浏览代码

YARN-2. Enhanced CapacityScheduler to account for CPU alongwith memory for multi-dimensional resource scheduling. Contributed by Arun C. Murthy.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1430682 13f79535-47bb-0310-9956-ffa450edef68
Arun Murthy 12 年之前
父节点
当前提交
4539263971
共有 58 个文件被更改,包括 1410 次插入442 次删除
  1. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
  2. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
  3. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
  4. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
  5. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
  6. 3 0
      hadoop-yarn-project/CHANGES.txt
  7. 41 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
  8. 17 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
  9. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  10. 20 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  11. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
  12. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java
  13. 33 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  14. 24 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
  15. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
  16. 12 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml
  17. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
  18. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  19. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
  20. 96 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DefaultResourceCalculator.java
  21. 180 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DominantResourceCalculator.java
  22. 0 73
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resource.java
  23. 149 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceCalculator.java
  24. 105 27
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java
  25. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  26. 8 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  27. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
  28. 19 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
  29. 40 28
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java
  30. 38 13
      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
  31. 35 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
  32. 10 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.java
  33. 107 70
      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
  34. 31 23
      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
  35. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
  36. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
  37. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
  38. 28 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
  39. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
  40. 150 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Resources.java
  41. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingAlgorithms.java
  42. 10 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  43. 5 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
  44. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationACLs.java
  45. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
  46. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java
  47. 10 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
  48. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
  49. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java
  50. 49 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
  51. 60 38
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
  52. 10 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
  53. 38 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
  54. 30 30
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
  55. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
  56. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
  57. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
  58. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java

@@ -227,9 +227,9 @@ public class MRApp extends MRAppMaster {
           this.clusterInfo.getMaxContainerCapability());
           this.clusterInfo.getMaxContainerCapability());
     } else {
     } else {
       getContext().getClusterInfo().setMinContainerCapability(
       getContext().getClusterInfo().setMinContainerCapability(
-          BuilderUtils.newResource(1024));
+          BuilderUtils.newResource(1024, 1));
       getContext().getClusterInfo().setMaxContainerCapability(
       getContext().getClusterInfo().setMaxContainerCapability(
-          BuilderUtils.newResource(10240));
+          BuilderUtils.newResource(10240, 1));
     }
     }
   }
   }
 
 

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java

@@ -207,9 +207,9 @@ public class MRAppBenchmark {
                 RegisterApplicationMasterResponse response =
                 RegisterApplicationMasterResponse response =
                     Records.newRecord(RegisterApplicationMasterResponse.class);
                     Records.newRecord(RegisterApplicationMasterResponse.class);
                 response.setMinimumResourceCapability(BuilderUtils
                 response.setMinimumResourceCapability(BuilderUtils
-                  .newResource(1024));
+                  .newResource(1024, 1));
                 response.setMaximumResourceCapability(BuilderUtils
                 response.setMaximumResourceCapability(BuilderUtils
-                  .newResource(10240));
+                  .newResource(10240, 1));
                 return response;
                 return response;
               }
               }
 
 

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java

@@ -1142,7 +1142,7 @@ public class TestRMContainerAllocator {
     }
     }
     TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId,
     TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId,
         taskAttemptId);
         taskAttemptId);
-    Resource containerNeed = BuilderUtils.newResource(memory);
+    Resource containerNeed = BuilderUtils.newResource(memory, 1);
     if (earlierFailedAttempt) {
     if (earlierFailedAttempt) {
       return ContainerRequestEvent
       return ContainerRequestEvent
           .createContainerRequestEventForFailedContainer(attemptId,
           .createContainerRequestEventForFailedContainer(attemptId,
@@ -1225,8 +1225,8 @@ public class TestRMContainerAllocator {
       when(context.getApplicationAttemptId()).thenReturn(appAttemptId);
       when(context.getApplicationAttemptId()).thenReturn(appAttemptId);
       when(context.getJob(isA(JobId.class))).thenReturn(job);
       when(context.getJob(isA(JobId.class))).thenReturn(job);
       when(context.getClusterInfo()).thenReturn(
       when(context.getClusterInfo()).thenReturn(
-          new ClusterInfo(BuilderUtils.newResource(1024), BuilderUtils
-              .newResource(10240)));
+          new ClusterInfo(BuilderUtils.newResource(1024, 1), BuilderUtils
+              .newResource(10240, 1)));
       when(context.getEventHandler()).thenReturn(new EventHandler() {
       when(context.getEventHandler()).thenReturn(new EventHandler() {
         @Override
         @Override
         public void handle(Event event) {
         public void handle(Event event) {
@@ -1299,12 +1299,12 @@ public class TestRMContainerAllocator {
 
 
     @Override
     @Override
     protected Resource getMinContainerCapability() {
     protected Resource getMinContainerCapability() {
-      return BuilderUtils.newResource(1024);
+      return BuilderUtils.newResource(1024, 1);
     }
     }
 
 
     @Override
     @Override
     protected Resource getMaxContainerCapability() {
     protected Resource getMaxContainerCapability() {
-      return BuilderUtils.newResource(10240);
+      return BuilderUtils.newResource(10240, 1);
     }
     }
 
 
     public void sendRequest(ContainerRequestEvent req) {
     public void sendRequest(ContainerRequestEvent req) {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java

@@ -197,7 +197,7 @@ public class TestTaskAttempt{
     conf.setInt(MRJobConfig.MAP_MEMORY_MB, mapMemMb);
     conf.setInt(MRJobConfig.MAP_MEMORY_MB, mapMemMb);
     conf.setInt(MRJobConfig.REDUCE_MEMORY_MB, reduceMemMb);
     conf.setInt(MRJobConfig.REDUCE_MEMORY_MB, reduceMemMb);
     app.setClusterInfo(new ClusterInfo(BuilderUtils
     app.setClusterInfo(new ClusterInfo(BuilderUtils
-        .newResource(minContainerSize), BuilderUtils.newResource(10240)));
+        .newResource(minContainerSize, 1), BuilderUtils.newResource(10240, 1)));
 
 
     Job job = app.submit(conf);
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     app.waitForState(job, JobState.RUNNING);

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java

@@ -99,8 +99,8 @@ public class TestLocalContainerAllocator {
       when(ctx.getApplicationAttemptId()).thenReturn(attemptId);
       when(ctx.getApplicationAttemptId()).thenReturn(attemptId);
       when(ctx.getJob(isA(JobId.class))).thenReturn(job);
       when(ctx.getJob(isA(JobId.class))).thenReturn(job);
       when(ctx.getClusterInfo()).thenReturn(
       when(ctx.getClusterInfo()).thenReturn(
-          new ClusterInfo(BuilderUtils.newResource(1024), BuilderUtils
-              .newResource(10240)));
+          new ClusterInfo(BuilderUtils.newResource(1024, 1), BuilderUtils
+              .newResource(10240, 1)));
       when(ctx.getEventHandler()).thenReturn(eventHandler);
       when(ctx.getEventHandler()).thenReturn(eventHandler);
       return ctx;
       return ctx;
     }
     }

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -32,6 +32,9 @@ Release 2.0.3-alpha - Unreleased
 
 
     YARN-286. Add a YARN ApplicationClassLoader. (tomwhite)
     YARN-286. Add a YARN ApplicationClassLoader. (tomwhite)
 
 
+    YARN-2. Enhanced CapacityScheduler to account for CPU alongwith memory for
+    multi-dimensional resource scheduling. (acmurthy)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     YARN-223. Update process tree instead of getting new process trees.
     YARN-223. Update process tree instead of getting new process trees.

+ 41 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.api.records;
 package org.apache.hadoop.yarn.api.records;
 
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 
 
@@ -54,11 +55,43 @@ public abstract class Resource implements Comparable<Resource> {
   @Stable
   @Stable
   public abstract void setMemory(int memory);
   public abstract void setMemory(int memory);
 
 
+
+  /**
+   * Get <em>number of virtual cpu cores</em> of the resource.
+   * 
+   * We refer to <em>virtual cores</em> to clarify that these represent
+   * <em>normalized</em> cores which may have a m:n relationship w.r.t
+   * physical cores available on the compute nodes. Furthermore, they also 
+   * represent <em>idealized</em> cores since the cluster might be composed
+   * of <em>heterogenous</em> nodes.
+   *   
+   * @return <em>num of virtual cpu cores</em> of the resource
+   */
+  @Public
+  @Evolving
+  public abstract int getVirtualCores();
+  
+  /**
+   * Set <em>number of virtual cpu cores</em> of the resource.
+   * 
+   * We refer to <em>virtual cores</em> to clarify that these represent
+   * <em>normalized</em> cores which may have a m:n relationship w.r.t
+   * physical cores available on the compute nodes. Furthermore, they also 
+   * represent <em>idealized</em> cores since the cluster might be composed
+   * of <em>heterogenous</em> nodes.
+   *   
+   * @param vCores <em>number of virtual cpu cores</em> of the resource
+   */
+  @Public
+  @Evolving
+  public abstract void setVirtualCores(int vCores);
+
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + getMemory();
+    final int prime = 263167;
+    int result = 3571;
+    result = 939769357 + getMemory(); // prime * result = 939769357 initially
+    result = prime * result + getVirtualCores();
     return result;
     return result;
   }
   }
 
 
@@ -68,16 +101,18 @@ public abstract class Resource implements Comparable<Resource> {
       return true;
       return true;
     if (obj == null)
     if (obj == null)
       return false;
       return false;
-    if (getClass() != obj.getClass())
+    if (!(obj instanceof Resource))
       return false;
       return false;
     Resource other = (Resource) obj;
     Resource other = (Resource) obj;
-    if (getMemory() != other.getMemory())
+    if (getMemory() != other.getMemory() || 
+        getVirtualCores() != other.getVirtualCores()) {
       return false;
       return false;
+    }
     return true;
     return true;
   }
   }
 
 
   @Override
   @Override
   public String toString() {
   public String toString() {
-    return "memory: " + getMemory();
+    return "<memory:" + getMemory() + ", vCores:" + getVirtualCores() + ">";
   }
   }
 }
 }

+ 17 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java

@@ -63,9 +63,25 @@ public class ResourcePBImpl extends Resource {
     builder.setMemory((memory));
     builder.setMemory((memory));
   }
   }
 
 
+  @Override
+  public int getVirtualCores() {
+    ResourceProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getVirtualCores());
+  }
+
+  @Override
+  public void setVirtualCores(int vCores) {
+    maybeInitBuilder();
+    builder.setVirtualCores((vCores));
+  }
+
   @Override
   @Override
   public int compareTo(Resource other) {
   public int compareTo(Resource other) {
-    return this.getMemory() - other.getMemory();
+    int diff = this.getMemory() - other.getMemory();
+    if (diff == 0) {
+      diff = this.getVirtualCores() - other.getVirtualCores();
+    }
+    return diff;
   }
   }
   
   
   
   

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto

@@ -48,6 +48,7 @@ message ContainerIdProto {
 
 
 message ResourceProto {
 message ResourceProto {
   optional int32 memory = 1;
   optional int32 memory = 1;
+  optional int32 virtual_cores = 2;
 }
 }
 
 
 message PriorityProto {
 message PriorityProto {

+ 20 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -29,10 +29,9 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Joiner;
-import com.google.common.base.Splitter;
 
 
 public class YarnConfiguration extends Configuration {
 public class YarnConfiguration extends Configuration {
-  private static final Splitter ADDR_SPLITTER = Splitter.on(':').trimResults();
+
   private static final Joiner JOINER = Joiner.on("");
   private static final Joiner JOINER = Joiner.on("");
 
 
   private static final String YARN_DEFAULT_XML_FILE = "yarn-default.xml";
   private static final String YARN_DEFAULT_XML_FILE = "yarn-default.xml";
@@ -111,15 +110,21 @@ public class YarnConfiguration extends Configuration {
   public static final String DEFAULT_RM_SCHEDULER_ADDRESS = "0.0.0.0:" +
   public static final String DEFAULT_RM_SCHEDULER_ADDRESS = "0.0.0.0:" +
     DEFAULT_RM_SCHEDULER_PORT;
     DEFAULT_RM_SCHEDULER_PORT;
 
 
-  /** Miniumum memory request grant-able by the RM scheduler. */
+  /** Miniumum request grant-able by the RM scheduler. */
   public static final String RM_SCHEDULER_MINIMUM_ALLOCATION_MB =
   public static final String RM_SCHEDULER_MINIMUM_ALLOCATION_MB =
     YARN_PREFIX + "scheduler.minimum-allocation-mb";
     YARN_PREFIX + "scheduler.minimum-allocation-mb";
   public static final int DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB = 1024;
   public static final int DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB = 1024;
+  public static final String RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES =
+      YARN_PREFIX + "scheduler.minimum-allocation-vcores";
+    public static final int DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES = 1;
 
 
-  /** Maximum memory request grant-able by the RM scheduler. */
+  /** Maximum request grant-able by the RM scheduler. */
   public static final String RM_SCHEDULER_MAXIMUM_ALLOCATION_MB =
   public static final String RM_SCHEDULER_MAXIMUM_ALLOCATION_MB =
     YARN_PREFIX + "scheduler.maximum-allocation-mb";
     YARN_PREFIX + "scheduler.maximum-allocation-mb";
   public static final int DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB = 8192;
   public static final int DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB = 8192;
+  public static final String RM_SCHEDULER_MAXIMUM_ALLOCATION_CORES =
+      YARN_PREFIX + "scheduler.maximum-allocation-vcores";
+  public static final int DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_CORES = 32;
 
 
   /** Number of threads to handle scheduler interface.*/
   /** Number of threads to handle scheduler interface.*/
   public static final String RM_SCHEDULER_CLIENT_THREAD_COUNT =
   public static final String RM_SCHEDULER_CLIENT_THREAD_COUNT =
@@ -408,10 +413,20 @@ public class YarnConfiguration extends Configuration {
   public static final String NM_PMEM_MB = NM_PREFIX + "resource.memory-mb";
   public static final String NM_PMEM_MB = NM_PREFIX + "resource.memory-mb";
   public static final int DEFAULT_NM_PMEM_MB = 8 * 1024;
   public static final int DEFAULT_NM_PMEM_MB = 8 * 1024;
 
 
+  /** Conversion ratio for physical memory to virtual memory. */
   public static final String NM_VMEM_PMEM_RATIO =
   public static final String NM_VMEM_PMEM_RATIO =
     NM_PREFIX + "vmem-pmem-ratio";
     NM_PREFIX + "vmem-pmem-ratio";
   public static final float DEFAULT_NM_VMEM_PMEM_RATIO = 2.1f;
   public static final float DEFAULT_NM_VMEM_PMEM_RATIO = 2.1f;
   
   
+  /** Number of Physical CPU Cores which can be allocated for containers.*/
+  public static final String NM_VCORES = NM_PREFIX + "resource.cpu-cores";
+  public static final int DEFAULT_NM_VCORES = 8;
+
+  /** Conversion ratio for physical cores to virtual cores. */
+  public static final String NM_VCORES_PCORES_RATIO =
+      NM_PREFIX + "vcores-pcores-ratio";
+  public static final float DEFAULT_NM_VCORES_PCORES_RATIO = 2.0f;
+  
   /** NM Webapp address.**/
   /** NM Webapp address.**/
   public static final String NM_WEBAPP_ADDRESS = NM_PREFIX + "webapp.address";
   public static final String NM_WEBAPP_ADDRESS = NM_PREFIX + "webapp.address";
   public static final int DEFAULT_NM_WEBAPP_PORT = 8042;
   public static final int DEFAULT_NM_WEBAPP_PORT = 8042;
@@ -675,4 +690,5 @@ public class YarnConfiguration extends Configuration {
   public static String getRMWebAppURL(Configuration conf) {
   public static String getRMWebAppURL(Configuration conf) {
     return JOINER.join("http://", getRMWebAppHostAndPort(conf));
     return JOINER.join("http://", getRMWebAppHostAndPort(conf));
   }
   }
+  
 }
 }

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java

@@ -106,6 +106,7 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
     out.writeUTF(this.nmHostAddr);
     out.writeUTF(this.nmHostAddr);
     out.writeUTF(this.appSubmitter);
     out.writeUTF(this.appSubmitter);
     out.writeInt(this.resource.getMemory());
     out.writeInt(this.resource.getMemory());
+    out.writeInt(this.resource.getVirtualCores());
     out.writeLong(this.expiryTimeStamp);
     out.writeLong(this.expiryTimeStamp);
     out.writeInt(this.masterKeyId);
     out.writeInt(this.masterKeyId);
   }
   }
@@ -120,7 +121,9 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
         .readInt());
         .readInt());
     this.nmHostAddr = in.readUTF();
     this.nmHostAddr = in.readUTF();
     this.appSubmitter = in.readUTF();
     this.appSubmitter = in.readUTF();
-    this.resource = BuilderUtils.newResource(in.readInt());
+    int memory = in.readInt();
+    int vCores = in.readInt();
+    this.resource = BuilderUtils.newResource(memory, vCores);
     this.expiryTimeStamp = in.readLong();
     this.expiryTimeStamp = in.readLong();
     this.masterKeyId = in.readInt();
     this.masterKeyId = in.readInt();
   }
   }

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java

@@ -371,9 +371,10 @@ public class BuilderUtils {
     return report;
     return report;
   }
   }
 
 
-  public static Resource newResource(int memory) {
+  public static Resource newResource(int memory, int vCores) {
     Resource resource = recordFactory.newRecordInstance(Resource.class);
     Resource resource = recordFactory.newRecordInstance(Resource.class);
     resource.setMemory(memory);
     resource.setMemory(memory);
+    resource.setVirtualCores(vCores);
     return resource;
     return resource;
   }
   }
 
 

+ 33 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -189,7 +189,7 @@
   </property>
   </property>
 
 
   <property>
   <property>
-    <description>The minimum allocation size for every container request at the RM,
+    <description>The minimum allocation for every container request at the RM,
     in MBs. Memory requests lower than this won't take effect,
     in MBs. Memory requests lower than this won't take effect,
     and the specified value will get allocated at minimum.</description>
     and the specified value will get allocated at minimum.</description>
     <name>yarn.scheduler.minimum-allocation-mb</name>
     <name>yarn.scheduler.minimum-allocation-mb</name>
@@ -197,13 +197,29 @@
   </property>
   </property>
 
 
   <property>
   <property>
-    <description>The maximum allocation size for every container request at the RM,
+    <description>The maximum allocation for every container request at the RM,
     in MBs. Memory requests higher than this won't take effect,
     in MBs. Memory requests higher than this won't take effect,
     and will get capped to this value.</description>
     and will get capped to this value.</description>
     <name>yarn.scheduler.maximum-allocation-mb</name>
     <name>yarn.scheduler.maximum-allocation-mb</name>
     <value>8192</value>
     <value>8192</value>
   </property>
   </property>
 
 
+  <property>
+    <description>The minimum allocation for every container request at the RM,
+    in terms of virtual CPU cores. Requests lower than this won't take effect,
+    and the specified value will get allocated the minimum.</description>
+    <name>yarn.scheduler.minimum-allocation-vcores</name>
+    <value>1</value>
+  </property>
+
+  <property>
+    <description>The maximum allocation for every container request at the RM,
+    in terms of virtual CPU cores. Requests higher than this won't take effect,
+    and will get capped to this value.</description>
+    <name>yarn.scheduler.maximum-allocation-vcores</name>
+    <value>32</value>
+  </property>
+
   <property>
   <property>
     <description>Enable RM to recover state after starting. If true, then 
     <description>Enable RM to recover state after starting. If true, then 
     yarn.resourcemanager.store.class must be specified</description>
     yarn.resourcemanager.store.class must be specified</description>
@@ -404,7 +420,6 @@
     <value>logs</value>
     <value>logs</value>
   </property>
   </property>
 
 
-
   <property>
   <property>
     <description>Amount of physical memory, in MB, that can be allocated 
     <description>Amount of physical memory, in MB, that can be allocated 
     for containers.</description>
     for containers.</description>
@@ -422,6 +437,21 @@
     <value>2.1</value>
     <value>2.1</value>
   </property>
   </property>
 
 
+  <property>
+    <description>Number of CPU cores that can be allocated 
+    for containers.</description>
+    <name>yarn.nodemanager.resource.cpu-cores</name>
+    <value>8</value>
+  </property>
+
+  <property>
+    <description>Ratio between virtual cores to physical cores when
+    allocating CPU resources to containers. 
+    </description>
+    <name>yarn.nodemanager.vcores-pcores-ratio</name>
+    <value>2</value>
+  </property>
+
   <property>
   <property>
     <description>NM Webapp address.</description>
     <description>NM Webapp address.</description>
     <name>yarn.nodemanager.webapp.address</name>
     <name>yarn.nodemanager.webapp.address</name>

+ 24 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java

@@ -106,14 +106,37 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     this.heartBeatInterval =
     this.heartBeatInterval =
         conf.getLong(YarnConfiguration.NM_TO_RM_HEARTBEAT_INTERVAL_MS,
         conf.getLong(YarnConfiguration.NM_TO_RM_HEARTBEAT_INTERVAL_MS,
             YarnConfiguration.DEFAULT_NM_TO_RM_HEARTBEAT_INTERVAL_MS);
             YarnConfiguration.DEFAULT_NM_TO_RM_HEARTBEAT_INTERVAL_MS);
-    int memoryMb = conf.getInt(YarnConfiguration.NM_PMEM_MB, YarnConfiguration.DEFAULT_NM_PMEM_MB);
+    int memoryMb = 
+        conf.getInt(
+            YarnConfiguration.NM_PMEM_MB, YarnConfiguration.DEFAULT_NM_PMEM_MB);
+    float vMemToPMem =             
+        conf.getFloat(
+            YarnConfiguration.NM_VMEM_PMEM_RATIO, 
+            YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO); 
+    int virtualMemoryMb = (int)Math.ceil(memoryMb * vMemToPMem);
+    
+    int cpuCores =
+        conf.getInt(
+            YarnConfiguration.NM_VCORES, YarnConfiguration.DEFAULT_NM_VCORES);
+    float vCoresToPCores =             
+        conf.getFloat(
+            YarnConfiguration.NM_VCORES_PCORES_RATIO, 
+            YarnConfiguration.DEFAULT_NM_VCORES_PCORES_RATIO); 
+    int virtualCores = (int)Math.ceil(cpuCores * vCoresToPCores); 
+
     this.totalResource = recordFactory.newRecordInstance(Resource.class);
     this.totalResource = recordFactory.newRecordInstance(Resource.class);
     this.totalResource.setMemory(memoryMb);
     this.totalResource.setMemory(memoryMb);
+    this.totalResource.setVirtualCores(virtualCores);
     metrics.addResource(totalResource);
     metrics.addResource(totalResource);
     this.tokenKeepAliveEnabled = isTokenKeepAliveEnabled(conf);
     this.tokenKeepAliveEnabled = isTokenKeepAliveEnabled(conf);
     this.tokenRemovalDelayMs =
     this.tokenRemovalDelayMs =
         conf.getInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS,
         conf.getInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS,
             YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
             YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
+    
+    LOG.info("Initialized nodemanager for " + nodeId + ":" +
+    		" physical-memory=" + memoryMb + " virtual-memory=" + virtualMemoryMb +
+    		" physical-cores=" + cpuCores + " virtual-cores=" + virtualCores);
+    
     super.init(conf);
     super.init(conf);
   }
   }
 
 

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java

@@ -574,7 +574,7 @@ public class TestContainer {
       when(ctxt.getUser()).thenReturn(this.user);
       when(ctxt.getUser()).thenReturn(this.user);
       when(ctxt.getContainerId()).thenReturn(cId);
       when(ctxt.getContainerId()).thenReturn(cId);
 
 
-      Resource resource = BuilderUtils.newResource(1024);
+      Resource resource = BuilderUtils.newResource(1024, 1);
       when(ctxt.getResource()).thenReturn(resource);
       when(ctxt.getResource()).thenReturn(resource);
 
 
       if (withLocalRes) {
       if (withLocalRes) {

+ 12 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml

@@ -18,6 +18,18 @@
     </description>
     </description>
   </property>
   </property>
 
 
+  <property>
+    <name>yarn.scheduler.capacity.resource-calculator</name>
+    <value>org.apache.hadoop.yarn.server.resourcemanager.resource.DefaultResourceCalculator</value>
+    <description>
+      The ResourceCalculator implementation to be used to compare 
+      Resources in the scheduler.
+      The default i.e. DefaultResourceCalculator only uses Memory while
+      DominantResourceCalculator uses dominant-resource to compare 
+      multi-dimensional resources such as Memory, CPU etc.
+    </description>
+  </property>
+
   <property>
   <property>
     <name>yarn.scheduler.capacity.root.queues</name>
     <name>yarn.scheduler.capacity.root.queues</name>
     <value>default</value>
     <value>default</value>

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

@@ -297,7 +297,7 @@ public class ApplicationMasterService extends AbstractService implements
         for(RMNode rmNode: updatedNodes) {
         for(RMNode rmNode: updatedNodes) {
           SchedulerNodeReport schedulerNodeReport =  
           SchedulerNodeReport schedulerNodeReport =  
               rScheduler.getNodeReport(rmNode.getNodeID());
               rScheduler.getNodeReport(rmNode.getNodeID());
-          Resource used = BuilderUtils.newResource(0);
+          Resource used = BuilderUtils.newResource(0, 0);
           int numContainers = 0;
           int numContainers = 0;
           if (schedulerNodeReport != null) {
           if (schedulerNodeReport != null) {
             used = schedulerNodeReport.getUsedResource();
             used = schedulerNodeReport.getUsedResource();

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

@@ -441,7 +441,7 @@ public class ClientRMService extends AbstractService implements
   private NodeReport createNodeReports(RMNode rmNode) {    
   private NodeReport createNodeReports(RMNode rmNode) {    
     SchedulerNodeReport schedulerNodeReport = 
     SchedulerNodeReport schedulerNodeReport = 
         scheduler.getNodeReport(rmNode.getNodeID());
         scheduler.getNodeReport(rmNode.getNodeID());
-    Resource used = BuilderUtils.newResource(0);
+    Resource used = BuilderUtils.newResource(0, 0);
     int numContainers = 0;
     int numContainers = 0;
     if (schedulerNodeReport != null) {
     if (schedulerNodeReport != null) {
       used = schedulerNodeReport.getUsedResource();
       used = schedulerNodeReport.getUsedResource();

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

@@ -193,7 +193,7 @@ public class ResourceTrackerService extends AbstractService implements
 
 
     LOG.info("NodeManager from node " + host + "(cmPort: " + cmPort
     LOG.info("NodeManager from node " + host + "(cmPort: " + cmPort
         + " httpPort: " + httpPort + ") " + "registered with capability: "
         + " httpPort: " + httpPort + ") " + "registered with capability: "
-        + capability.getMemory() + ", assigned nodeId " + nodeId);
+        + capability + ", assigned nodeId " + nodeId);
 
 
     regResponse.setNodeAction(NodeAction.NORMAL);
     regResponse.setNodeAction(NodeAction.NORMAL);
     response.setRegistrationResponse(regResponse);
     response.setRegistrationResponse(regResponse);

+ 96 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DefaultResourceCalculator.java

@@ -0,0 +1,96 @@
+/**
+* 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.resource;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+
+public class DefaultResourceCalculator extends ResourceCalculator {
+
+  Resource clusterResource;
+  
+  @Override
+  public int compare(Resource unused, Resource lhs, Resource rhs) {
+    // Only consider memory
+    return lhs.getMemory() - rhs.getMemory();
+  }
+
+  @Override
+  public int computeAvailableContainers(Resource available, Resource required) {
+    // Only consider memory
+    return available.getMemory() / required.getMemory();
+  }
+
+  @Override
+  public float divide(Resource unused, 
+      Resource numerator, Resource denominator) {
+    return ratio(numerator, denominator);
+  }
+
+  @Override
+  public float ratio(Resource a, Resource b) {
+    return (float)a.getMemory() / b.getMemory();
+  }
+
+  @Override
+  public Resource divideAndCeil(Resource numerator, int denominator) {
+    return Resources.createResource(
+        divideAndCeil(numerator.getMemory(), denominator));
+  }
+
+  @Override
+  public Resource normalize(Resource r, Resource minimumResource) {
+    return Resources.createResource(
+        roundUp(
+            Math.max(r.getMemory(), minimumResource.getMemory()),
+            minimumResource.getMemory())
+        );
+  }
+
+  @Override
+  public Resource roundUp(Resource r, Resource minimumResource) {
+    return Resources.createResource(
+        roundUp(r.getMemory(),minimumResource.getMemory())
+        );
+  }
+
+  @Override
+  public Resource roundDown(Resource r, Resource stepFactor) {
+    return Resources.createResource(
+        roundDown(r.getMemory(), stepFactor.getMemory()));
+  }
+
+  @Override
+  public Resource multiplyAndNormalizeUp(Resource r, double by,
+      Resource stepFactor) {
+    return Resources.createResource(
+        roundUp((int)(r.getMemory() * by + 0.5), stepFactor.getMemory())
+        );
+  }
+
+  @Override
+  public Resource multiplyAndNormalizeDown(Resource r, double by,
+      Resource stepFactor) {
+    return Resources.createResource(
+        roundDown(
+            (int)(r.getMemory() * by), 
+            stepFactor.getMemory()
+            )
+        );
+  }
+
+}

+ 180 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DominantResourceCalculator.java

@@ -0,0 +1,180 @@
+/**
+* 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.resource;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+
+/**
+ * A {@link ResourceCalculator} which uses the concept of  
+ * <em>dominant resource</em> to compare multi-dimensional resources.
+ *
+ * Essentially the idea is that the in a multi-resource environment, 
+ * the resource allocation should be determined by the dominant share 
+ * of an entity (user or queue), which is the maximum share that the 
+ * entity has been allocated of any resource. 
+ * 
+ * In a nutshell, it seeks to maximize the minimum dominant share across 
+ * all entities. 
+ * 
+ * For example, if user A runs CPU-heavy tasks and user B runs
+ * memory-heavy tasks, it attempts to equalize CPU share of user A 
+ * with Memory-share of user B. 
+ * 
+ * In the single resource case, it reduces to max-min fairness for that resource.
+ * 
+ * See the Dominant Resource Fairness paper for more details:
+ * www.cs.berkeley.edu/~matei/papers/2011/nsdi_drf.pdf
+ */
+public class DominantResourceCalculator extends ResourceCalculator {
+  
+  @Override
+  public int compare(Resource clusterResource, Resource lhs, Resource rhs) {
+    
+    if (lhs.equals(rhs)) {
+      return 0;
+    }
+    
+    float l = getResourceAsValue(clusterResource, lhs, true);
+    float r = getResourceAsValue(clusterResource, rhs, true);
+    
+    if (l < r) {
+      return -1;
+    } else if (l > r) {
+      return 1;
+    } else {
+      l = getResourceAsValue(clusterResource, lhs, false);
+      r = getResourceAsValue(clusterResource, rhs, false);
+      if (l < r) {
+        return -1;
+      } else if (l > r) {
+        return 1;
+      }
+    }
+    
+    return 0;
+  }
+
+  /**
+   * Use 'dominant' for now since we only have 2 resources - gives us a slight
+   * performance boost.
+   * 
+   * Once we add more resources, we'll need a more complicated (and slightly
+   * less performant algorithm).
+   */
+  protected float getResourceAsValue(
+      Resource clusterResource, Resource resource, boolean dominant) {
+    // Just use 'dominant' resource
+    return (dominant) ?
+        Math.max(
+            (float)resource.getMemory() / clusterResource.getMemory(), 
+            (float)resource.getVirtualCores() / clusterResource.getVirtualCores()
+            ) 
+        :
+          Math.min(
+              (float)resource.getMemory() / clusterResource.getMemory(), 
+              (float)resource.getVirtualCores() / clusterResource.getVirtualCores()
+              ); 
+  }
+  
+  @Override
+  public int computeAvailableContainers(Resource available, Resource required) {
+    return Math.min(
+        available.getMemory() / required.getMemory(), 
+        available.getVirtualCores() / required.getVirtualCores());
+  }
+
+  @Override
+  public float divide(Resource clusterResource, 
+      Resource numerator, Resource denominator) {
+    return 
+        getResourceAsValue(clusterResource, numerator, true) / 
+        getResourceAsValue(clusterResource, denominator, true);
+  }
+
+  @Override
+  public float ratio(Resource a, Resource b) {
+    return Math.max(
+        (float)a.getMemory()/b.getMemory(), 
+        (float)a.getVirtualCores()/b.getVirtualCores()
+        );
+  }
+
+  @Override
+  public Resource divideAndCeil(Resource numerator, int denominator) {
+    return Resources.createResource(
+        divideAndCeil(numerator.getMemory(), denominator),
+        divideAndCeil(numerator.getVirtualCores(), denominator)
+        );
+  }
+
+  @Override
+  public Resource normalize(Resource r, Resource minimumResource) {
+    return Resources.createResource(
+        roundUp(
+            Math.max(r.getMemory(), minimumResource.getMemory()), 
+            minimumResource.getMemory()),
+        roundUp(
+            Math.max(r.getVirtualCores(), minimumResource.getVirtualCores()),
+            minimumResource.getVirtualCores())
+        );
+  }
+
+  @Override
+  public Resource roundUp(Resource r, Resource minimumResource) {
+    return Resources.createResource(
+        roundUp(r.getMemory(), minimumResource.getMemory()), 
+        roundUp(r.getVirtualCores(), minimumResource.getVirtualCores())
+        );
+  }
+
+  @Override
+  public Resource roundDown(Resource r, Resource stepFactor) {
+    return Resources.createResource(
+        roundDown(r.getMemory(), stepFactor.getMemory()),
+        roundDown(r.getVirtualCores(), stepFactor.getVirtualCores())
+        );
+  }
+
+  @Override
+  public Resource multiplyAndNormalizeUp(Resource r, double by,
+      Resource stepFactor) {
+    return Resources.createResource(
+        roundUp(
+            (int)Math.ceil(r.getMemory() * by), stepFactor.getMemory()),
+        roundUp(
+            (int)Math.ceil(r.getVirtualCores() * by), 
+            stepFactor.getVirtualCores())
+        );
+  }
+
+  @Override
+  public Resource multiplyAndNormalizeDown(Resource r, double by,
+      Resource stepFactor) {
+    return Resources.createResource(
+        roundDown(
+            (int)(r.getMemory() * by), 
+            stepFactor.getMemory()
+            ),
+        roundDown(
+            (int)(r.getVirtualCores() * by), 
+            stepFactor.getVirtualCores()
+            )
+        );
+  }
+
+}

+ 0 - 73
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resource.java

@@ -1,73 +0,0 @@
-/**
-* 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.resource;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-
-@Private
-@Evolving
-public class Resource {
-  public static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);  
-  public static final org.apache.hadoop.yarn.api.records.Resource NONE = createResource(0);
-
-
-  public static org.apache.hadoop.yarn.api.records.Resource createResource(int memory) {
-    org.apache.hadoop.yarn.api.records.Resource resource = recordFactory.newRecordInstance(org.apache.hadoop.yarn.api.records.Resource.class);
-    resource.setMemory(memory);
-    return resource;
-  }
-  
-  public static void addResource(org.apache.hadoop.yarn.api.records.Resource lhs, 
-      org.apache.hadoop.yarn.api.records.Resource rhs) {
-    lhs.setMemory(lhs.getMemory() + rhs.getMemory());
-  }
-  
-  public static void subtractResource(org.apache.hadoop.yarn.api.records.Resource lhs, 
-      org.apache.hadoop.yarn.api.records.Resource rhs) {
-    lhs.setMemory(lhs.getMemory() - rhs.getMemory());
-  }
-  
-  public static boolean equals(org.apache.hadoop.yarn.api.records.Resource lhs,
-      org.apache.hadoop.yarn.api.records.Resource rhs) {
-    return lhs.getMemory() == rhs.getMemory();
-  }
-
-  public static boolean lessThan(org.apache.hadoop.yarn.api.records.Resource lhs,
-      org.apache.hadoop.yarn.api.records.Resource rhs) {
-    return lhs.getMemory() < rhs.getMemory();
-  }
-
-  public static boolean lessThanOrEqual(org.apache.hadoop.yarn.api.records.Resource lhs,
-      org.apache.hadoop.yarn.api.records.Resource rhs) {
-    return lhs.getMemory() <= rhs.getMemory();
-  }
-
-  public static boolean greaterThan(org.apache.hadoop.yarn.api.records.Resource lhs,
-      org.apache.hadoop.yarn.api.records.Resource rhs) {
-    return lhs.getMemory() > rhs.getMemory();
-  }
-
-  public static boolean greaterThanOrEqual(org.apache.hadoop.yarn.api.records.Resource lhs,
-      org.apache.hadoop.yarn.api.records.Resource rhs) {
-    return lhs.getMemory() >= rhs.getMemory();
-  }
-}

+ 149 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceCalculator.java

@@ -0,0 +1,149 @@
+/**
+* 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.resource;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+/**
+ * A set of {@link Resource} comparison and manipulation interfaces.
+ */
+@Private
+@Unstable
+public abstract class ResourceCalculator {
+
+  private static final Log LOG = LogFactory.getLog(ResourceCalculator.class);
+
+  public abstract int 
+  compare(Resource clusterResource, Resource lhs, Resource rhs);
+  
+  public static int divideAndCeil(int a, int b) {
+    if (b == 0) {
+      LOG.info("divideAndCeil called with a=" + a + " b=" + b);
+      return 0;
+    }
+    return (a + (b - 1)) / b;
+  }
+
+  public static int roundUp(int a, int b) {
+    return divideAndCeil(a, b) * b;
+  }
+
+  public static int roundDown(int a, int b) {
+    return (a / b) * b;
+  }
+
+  /**
+   * Compute the number of containers which can be allocated given
+   * <code>available</code> and <code>required</code> resources.
+   * 
+   * @param available available resources
+   * @param required required resources
+   * @return number of containers which can be allocated
+   */
+  public abstract int computeAvailableContainers(
+      Resource available, Resource required);
+
+  /**
+   * Multiply resource <code>r</code> by factor <code>by</code> 
+   * and normalize up using step-factor <code>stepFactor</code>.
+   * 
+   * @param r resource to be multiplied
+   * @param by multiplier
+   * @param stepFactor factor by which to normalize up 
+   * @return resulting normalized resource
+   */
+  public abstract Resource multiplyAndNormalizeUp(
+      Resource r, double by, Resource stepFactor);
+  
+  /**
+   * Multiply resource <code>r</code> by factor <code>by</code> 
+   * and normalize down using step-factor <code>stepFactor</code>.
+   * 
+   * @param r resource to be multiplied
+   * @param by multiplier
+   * @param stepFactor factor by which to normalize down 
+   * @return resulting normalized resource
+   */
+  public abstract Resource multiplyAndNormalizeDown(
+      Resource r, double by, Resource stepFactor);
+  
+  /**
+   * Normalize resource <code>r</code> given the base 
+   * <code>minimumResource</code>.
+   * 
+   * @param r resource
+   * @param minimumResource step-factor
+   * @return normalized resource
+   */
+  public abstract Resource normalize(Resource r, Resource minimumResource);
+
+  /**
+   * Round-up resource <code>r</code> given factor <code>stepFactor</code>.
+   * 
+   * @param r resource
+   * @param stepFactor step-factor
+   * @return rounded resource
+   */
+  public abstract Resource roundUp(Resource r, Resource stepFactor);
+  
+  /**
+   * Round-down resource <code>r</code> given factor <code>stepFactor</code>.
+   * 
+   * @param r resource
+   * @param stepFactor step-factor
+   * @return rounded resource
+   */
+  public abstract Resource roundDown(Resource r, Resource stepFactor);
+  
+  /**
+   * Divide resource <code>numerator</code> by resource <code>denominator</code>
+   * using specified policy (domination, average, fairness etc.); hence overall
+   * <code>clusterResource</code> is provided for context.
+   *  
+   * @param clusterResource cluster resources
+   * @param numerator numerator
+   * @param denominator denominator
+   * @return <code>numerator</code>/<code>denominator</code> 
+   *         using specific policy
+   */
+  public abstract float divide(
+      Resource clusterResource, Resource numerator, Resource denominator);
+  
+  /**
+   * Ratio of resource <code>a</code> to resource <code>b</code>.
+   * 
+   * @param a resource 
+   * @param b resource
+   * @return ratio of resource <code>a</code> to resource <code>b</code>
+   */
+  public abstract float ratio(Resource a, Resource b);
+
+  /**
+   * Divide-and-ceil <code>numerator</code> by <code>denominator</code>.
+   * 
+   * @param numerator numerator resource
+   * @param denominator denominator
+   * @return resultant resource
+   */
+  public abstract Resource divideAndCeil(Resource numerator, int denominator);
+  
+}

+ 105 - 27
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java

@@ -40,16 +40,35 @@ public class Resources {
       throw new RuntimeException("NONE cannot be modified!");
       throw new RuntimeException("NONE cannot be modified!");
     }
     }
 
 
+    @Override
+    public int getVirtualCores() {
+      return 0;
+    }
+
+    @Override
+    public void setVirtualCores(int cores) {
+      throw new RuntimeException("NONE cannot be modified!");
+    }
+
     @Override
     @Override
     public int compareTo(Resource o) {
     public int compareTo(Resource o) {
-      return (0 - o.getMemory());
+      int diff = 0 - o.getMemory();
+      if (diff == 0) {
+        diff = 0 - o.getVirtualCores();
+      }
+      return diff;
     }
     }
     
     
   };
   };
 
 
   public static Resource createResource(int memory) {
   public static Resource createResource(int memory) {
+    return createResource(memory, (memory > 0) ? 1 : 0);
+  }
+
+  public static Resource createResource(int memory, int cores) {
     Resource resource = Records.newRecord(Resource.class);
     Resource resource = Records.newRecord(Resource.class);
     resource.setMemory(memory);
     resource.setMemory(memory);
+    resource.setVirtualCores(cores);
     return resource;
     return resource;
   }
   }
 
 
@@ -58,11 +77,12 @@ public class Resources {
   }
   }
 
 
   public static Resource clone(Resource res) {
   public static Resource clone(Resource res) {
-    return createResource(res.getMemory());
+    return createResource(res.getMemory(), res.getVirtualCores());
   }
   }
 
 
   public static Resource addTo(Resource lhs, Resource rhs) {
   public static Resource addTo(Resource lhs, Resource rhs) {
     lhs.setMemory(lhs.getMemory() + rhs.getMemory());
     lhs.setMemory(lhs.getMemory() + rhs.getMemory());
+    lhs.setVirtualCores(lhs.getVirtualCores() + rhs.getVirtualCores());
     return lhs;
     return lhs;
   }
   }
 
 
@@ -72,6 +92,7 @@ public class Resources {
 
 
   public static Resource subtractFrom(Resource lhs, Resource rhs) {
   public static Resource subtractFrom(Resource lhs, Resource rhs) {
     lhs.setMemory(lhs.getMemory() - rhs.getMemory());
     lhs.setMemory(lhs.getMemory() - rhs.getMemory());
+    lhs.setVirtualCores(lhs.getVirtualCores() - rhs.getVirtualCores());
     return lhs;
     return lhs;
   }
   }
 
 
@@ -83,50 +104,107 @@ public class Resources {
     return subtract(NONE, resource);
     return subtract(NONE, resource);
   }
   }
 
 
-  public static Resource multiplyTo(Resource lhs, int by) {
-    lhs.setMemory(lhs.getMemory() * by);
+  public static Resource multiplyTo(Resource lhs, double by) {
+    lhs.setMemory((int)(lhs.getMemory() * by));
+    lhs.setVirtualCores((int)(lhs.getVirtualCores() * by));
     return lhs;
     return lhs;
   }
   }
 
 
-  public static Resource multiply(Resource lhs, int by) {
+  public static Resource multiply(Resource lhs, double by) {
     return multiplyTo(clone(lhs), by);
     return multiplyTo(clone(lhs), by);
   }
   }
   
   
-  /**
-   * Mutliply a resource by a {@code double}. Note that integral 
-   * resource quantites are subject to rounding during cast.
-   */
-  public static Resource multiply(Resource lhs, double by) {
+  public static Resource multiplyAndNormalizeUp(
+      ResourceCalculator calculator,Resource lhs, double by, Resource factor) {
+    return calculator.multiplyAndNormalizeUp(lhs, by, factor);
+  }
+  
+  public static Resource multiplyAndNormalizeDown(
+      ResourceCalculator calculator,Resource lhs, double by, Resource factor) {
+    return calculator.multiplyAndNormalizeDown(lhs, by, factor);
+  }
+  
+  public static Resource multiplyAndRoundDown(Resource lhs, double by) {
     Resource out = clone(lhs);
     Resource out = clone(lhs);
-    out.setMemory((int) (lhs.getMemory() * by));
+    out.setMemory((int)(lhs.getMemory() * by));
+    out.setVirtualCores((int)(lhs.getVirtualCores() * by));
     return out;
     return out;
   }
   }
-
+  
+  public static Resource normalize(
+      ResourceCalculator calculator, Resource lhs, Resource factor) {
+    return calculator.normalize(lhs, factor);
+  }
+  
+  public static Resource roundUp(
+      ResourceCalculator calculator, Resource lhs, Resource factor) {
+    return calculator.roundUp(lhs, factor);
+  }
+  
+  public static Resource roundDown(
+      ResourceCalculator calculator, Resource lhs, Resource factor) {
+    return calculator.roundDown(lhs, factor);
+  }
+  
+  public static float ratio(
+      ResourceCalculator resourceCalculator, Resource lhs, Resource rhs) {
+    return resourceCalculator.ratio(lhs, rhs);
+  }
+  
+  public static float divide(
+      ResourceCalculator resourceCalculator,
+      Resource clusterResource, Resource lhs, Resource rhs) {
+    return resourceCalculator.divide(clusterResource, lhs, rhs);
+  }
+  
+  public static Resource divideAndCeil(
+      ResourceCalculator resourceCalculator, Resource lhs, int rhs) {
+    return resourceCalculator.divideAndCeil(lhs, rhs);
+  }
+  
   public static boolean equals(Resource lhs, Resource rhs) {
   public static boolean equals(Resource lhs, Resource rhs) {
-    return lhs.getMemory() == rhs.getMemory();
+    return lhs.equals(rhs);
   }
   }
 
 
-  public static boolean lessThan(Resource lhs, Resource rhs) {
-    return lhs.getMemory() < rhs.getMemory();
+  public static boolean lessThan(
+      ResourceCalculator resourceCalculator, 
+      Resource clusterResource,
+      Resource lhs, Resource rhs) {
+    return (resourceCalculator.compare(clusterResource, lhs, rhs) < 0);
   }
   }
 
 
-  public static boolean lessThanOrEqual(Resource lhs, Resource rhs) {
-    return lhs.getMemory() <= rhs.getMemory();
+  public static boolean lessThanOrEqual(
+      ResourceCalculator resourceCalculator, 
+      Resource clusterResource,
+      Resource lhs, Resource rhs) {
+    return (resourceCalculator.compare(clusterResource, lhs, rhs) <= 0);
   }
   }
 
 
-  public static boolean greaterThan(Resource lhs, Resource rhs) {
-    return lhs.getMemory() > rhs.getMemory();
+  public static boolean greaterThan(
+      ResourceCalculator resourceCalculator,
+      Resource clusterResource,
+      Resource lhs, Resource rhs) {
+    return resourceCalculator.compare(clusterResource, lhs, rhs) > 0;
   }
   }
 
 
-  public static boolean greaterThanOrEqual(Resource lhs, Resource rhs) {
-    return lhs.getMemory() >= rhs.getMemory();
+  public static boolean greaterThanOrEqual(
+      ResourceCalculator resourceCalculator, 
+      Resource clusterResource,
+      Resource lhs, Resource rhs) {
+    return resourceCalculator.compare(clusterResource, lhs, rhs) >= 0;
   }
   }
   
   
-  public static Resource min(Resource lhs, Resource rhs) {
-    return (lhs.getMemory() < rhs.getMemory()) ? lhs : rhs;
-  }
-
-  public static Resource max(Resource lhs, Resource rhs) {
-    return (lhs.getMemory() > rhs.getMemory()) ? lhs : rhs;
+  public static Resource min(
+      ResourceCalculator resourceCalculator, 
+      Resource clusterResource,
+      Resource lhs, Resource rhs) {
+    return resourceCalculator.compare(clusterResource, lhs, rhs) <= 0 ? lhs : rhs;
+  }
+
+  public static Resource max(
+      ResourceCalculator resourceCalculator, 
+      Resource clusterResource,
+      Resource lhs, Resource rhs) {
+    return resourceCalculator.compare(clusterResource, lhs, rhs) >= 0 ? lhs : rhs;
   }
   }
 }
 }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java

@@ -207,8 +207,8 @@ public class RMAppImpl implements RMApp, Recoverable {
   private static final ApplicationResourceUsageReport
   private static final ApplicationResourceUsageReport
     DUMMY_APPLICATION_RESOURCE_USAGE_REPORT =
     DUMMY_APPLICATION_RESOURCE_USAGE_REPORT =
       BuilderUtils.newApplicationResourceUsageReport(-1, -1,
       BuilderUtils.newApplicationResourceUsageReport(-1, -1,
-          Resources.createResource(-1), Resources.createResource(-1),
-          Resources.createResource(-1));
+          Resources.createResource(-1, -1), Resources.createResource(-1, -1),
+          Resources.createResource(-1, -1));
   private static final int DUMMY_APPLICATION_ATTEMPT_NUMBER = -1;
   private static final int DUMMY_APPLICATION_ATTEMPT_NUMBER = -1;
   
   
   public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
   public RMAppImpl(ApplicationId applicationId, RMContext rmContext,

+ 8 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -591,8 +592,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     try {
     try {
       int numUsedContainers = 0;
       int numUsedContainers = 0;
       int numReservedContainers = 0;
       int numReservedContainers = 0;
-      int reservedResources = 0;
-      int currentConsumption = 0;
+      Resource currentConsumption = Resources.createResource(0, 0);
+      Resource reservedResources = Resources.createResource(0, 0);
+      
       SchedulerAppReport schedApp = 
       SchedulerAppReport schedApp = 
           scheduler.getSchedulerAppInfo(this.getAppAttemptId());
           scheduler.getSchedulerAppInfo(this.getAppAttemptId());
       Collection<RMContainer> liveContainers;
       Collection<RMContainer> liveContainers;
@@ -603,22 +605,21 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         if (liveContainers != null) {
         if (liveContainers != null) {
           numUsedContainers = liveContainers.size();
           numUsedContainers = liveContainers.size();
           for (RMContainer lc : liveContainers) {
           for (RMContainer lc : liveContainers) {
-            currentConsumption += lc.getContainer().getResource().getMemory();
+            Resources.addTo(currentConsumption, lc.getContainer().getResource());
           }
           }
         }
         }
         if (reservedContainers != null) {
         if (reservedContainers != null) {
           numReservedContainers = reservedContainers.size();
           numReservedContainers = reservedContainers.size();
           for (RMContainer rc : reservedContainers) {
           for (RMContainer rc : reservedContainers) {
-            reservedResources += rc.getContainer().getResource().getMemory();
+            Resources.addTo(reservedResources, rc.getContainer().getResource());
           }
           }
         }
         }
       }
       }
 
 
       return BuilderUtils.newApplicationResourceUsageReport(
       return BuilderUtils.newApplicationResourceUsageReport(
           numUsedContainers, numReservedContainers,
           numUsedContainers, numReservedContainers,
-          Resources.createResource(currentConsumption),
-          Resources.createResource(reservedResources),
-          Resources.createResource(currentConsumption + reservedResources));
+          currentConsumption, reservedResources,
+          Resources.add(currentConsumption, reservedResources));
     } finally {
     } finally {
       this.readLock.unlock();
       this.readLock.unlock();
     }
     }

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

@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 
 import static org.apache.hadoop.metrics2.lib.Interns.info;
 import static org.apache.hadoop.metrics2.lib.Interns.info;
-import static org.apache.hadoop.yarn.server.resourcemanager.resource.Resources.multiply;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashMap;
@@ -42,6 +41,7 @@ import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
@@ -325,7 +325,7 @@ public class QueueMetrics implements MetricsSource {
     allocatedContainers.incr(containers);
     allocatedContainers.incr(containers);
     aggregateContainersAllocated.incr(containers);
     aggregateContainersAllocated.incr(containers);
     allocatedMB.incr(res.getMemory() * containers);
     allocatedMB.incr(res.getMemory() * containers);
-    _decrPendingResources(containers, multiply(res, containers));
+    _decrPendingResources(containers, Resources.multiply(res, containers));
     QueueMetrics userMetrics = getUserMetrics(user);
     QueueMetrics userMetrics = getUserMetrics(user);
     if (userMetrics != null) {
     if (userMetrics != null) {
       userMetrics.allocateResources(user, containers, res);
       userMetrics.allocateResources(user, containers, res);

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

@@ -24,10 +24,13 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 
 
 /**
 /**
  * Utilities shared by schedulers. 
  * Utilities shared by schedulers. 
@@ -81,32 +84,31 @@ public class SchedulerUtils {
   /**
   /**
    * Utility method to normalize a list of resource requests, by insuring that
    * Utility method to normalize a list of resource requests, by insuring that
    * the memory for each request is a multiple of minMemory and is not zero.
    * the memory for each request is a multiple of minMemory and is not zero.
-   *
-   * @param asks
-   *          a list of resource requests.
-   * @param minMemory
-   *          the configured minimum memory allocation.
    */
    */
-  public static void normalizeRequests(List<ResourceRequest> asks,
-      int minMemory) {
+  public static void normalizeRequests(
+      List<ResourceRequest> asks,
+      ResourceCalculator resourceCalculator, 
+      Resource clusterResource,
+      Resource minimumResource) {
     for (ResourceRequest ask : asks) {
     for (ResourceRequest ask : asks) {
-      normalizeRequest(ask, minMemory);
+      normalizeRequest(
+          ask, resourceCalculator, clusterResource, minimumResource);
     }
     }
   }
   }
 
 
   /**
   /**
    * Utility method to normalize a resource request, by insuring that the
    * Utility method to normalize a resource request, by insuring that the
    * requested memory is a multiple of minMemory and is not zero.
    * requested memory is a multiple of minMemory and is not zero.
-   *
-   * @param ask
-   *          the resource request.
-   * @param minMemory
-   *          the configured minimum memory allocation.
    */
    */
-  public static void normalizeRequest(ResourceRequest ask, int minMemory) {
-    int memory = Math.max(ask.getCapability().getMemory(), minMemory);
-    ask.getCapability().setMemory(
-        minMemory * ((memory / minMemory) + (memory % minMemory > 0 ? 1 : 0)));
+  public static void normalizeRequest(
+      ResourceRequest ask, 
+      ResourceCalculator resourceCalculator, 
+      Resource clusterResource,
+      Resource minimumResource) {
+    Resource normalized = 
+        Resources.normalize(
+            resourceCalculator, ask.getCapability(), minimumResource);
+    ask.setCapability(normalized);
   }
   }
 
 
 }
 }

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

@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 
 import org.apache.hadoop.yarn.Lock;
 import org.apache.hadoop.yarn.Lock;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 
 
 class CSQueueUtils {
 class CSQueueUtils {
@@ -51,15 +52,19 @@ class CSQueueUtils {
     return (parentAbsMaxCapacity * maximumCapacity);
     return (parentAbsMaxCapacity * maximumCapacity);
   }
   }
 
 
-  public static int computeMaxActiveApplications(Resource clusterResource,
-      Resource minimumAllocation, float maxAMResourcePercent, 
-      float absoluteMaxCapacity) {
-    return 
+  public static int computeMaxActiveApplications(
+      ResourceCalculator calculator,
+      Resource clusterResource, Resource minimumAllocation, 
+      float maxAMResourcePercent, float absoluteMaxCapacity) {
+    return
         Math.max(
         Math.max(
             (int)Math.ceil(
             (int)Math.ceil(
-                     ((float)clusterResource.getMemory() / 
-                         minimumAllocation.getMemory()) * 
-                     maxAMResourcePercent * absoluteMaxCapacity), 
+                Resources.ratio(
+                    calculator, 
+                    clusterResource, 
+                    minimumAllocation) * 
+                    maxAMResourcePercent * absoluteMaxCapacity
+                ), 
             1);
             1);
   }
   }
 
 
@@ -73,36 +78,43 @@ class CSQueueUtils {
   
   
   @Lock(CSQueue.class)
   @Lock(CSQueue.class)
   public static void updateQueueStatistics(
   public static void updateQueueStatistics(
+      final ResourceCalculator calculator,
       final CSQueue childQueue, final CSQueue parentQueue, 
       final CSQueue childQueue, final CSQueue parentQueue, 
       final Resource clusterResource, final Resource minimumAllocation) {
       final Resource clusterResource, final Resource minimumAllocation) {
-    final int clusterMemory = clusterResource.getMemory();
-    final int usedMemory = childQueue.getUsedResources().getMemory();
+    Resource queueLimit = Resources.none();
+    Resource usedResources = childQueue.getUsedResources();
     
     
-    float queueLimit = 0.0f;
     float absoluteUsedCapacity = 0.0f;
     float absoluteUsedCapacity = 0.0f;
     float usedCapacity = 0.0f;
     float usedCapacity = 0.0f;
-    if (clusterMemory > 0) {
-      queueLimit = clusterMemory * childQueue.getAbsoluteCapacity();
-      absoluteUsedCapacity = ((float)usedMemory / (float)clusterMemory);
-      usedCapacity = (queueLimit == 0) ? 0 : (usedMemory / queueLimit);
+
+    if (Resources.greaterThan(
+        calculator, clusterResource, clusterResource, Resources.none())) {
+      queueLimit = 
+          Resources.multiply(clusterResource, childQueue.getAbsoluteCapacity());
+      absoluteUsedCapacity = 
+          Resources.divide(calculator, clusterResource, 
+              usedResources, clusterResource);
+      usedCapacity = 
+          Resources.equals(queueLimit, Resources.none()) ? 0 :
+          Resources.divide(calculator, clusterResource, 
+              usedResources, queueLimit);
     }
     }
     
     
     childQueue.setUsedCapacity(usedCapacity);
     childQueue.setUsedCapacity(usedCapacity);
     childQueue.setAbsoluteUsedCapacity(absoluteUsedCapacity);
     childQueue.setAbsoluteUsedCapacity(absoluteUsedCapacity);
     
     
-    int available = 
-        Math.max((roundUp(minimumAllocation, (int)queueLimit) - usedMemory), 0); 
+    Resource available = 
+        Resources.roundUp(
+            calculator, 
+            Resources.subtract(queueLimit, usedResources), 
+            minimumAllocation);
     childQueue.getMetrics().setAvailableResourcesToQueue(
     childQueue.getMetrics().setAvailableResourcesToQueue(
-        Resources.createResource(available));
-  }
-
-  public static int roundUp(Resource minimumAllocation, int memory) {
-    int minMemory = minimumAllocation.getMemory();
-    return LeafQueue.divideAndCeil(memory, minMemory) * minMemory; 
-  }
-
-  public static int roundDown(Resource minimumAllocation, int memory) {
-    int minMemory = minimumAllocation.getMemory();
-    return (memory / minMemory) * minMemory;
-  }
+        Resources.max(
+            calculator, 
+            clusterResource, 
+            available, 
+            Resources.none()
+            )
+        );
+   }
 }
 }

+ 38 - 13
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

@@ -50,6 +50,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
@@ -141,6 +142,8 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
 
 
   private boolean initialized = false;
   private boolean initialized = false;
 
 
+  private ResourceCalculator calculator;
+  
   public CapacityScheduler() {}
   public CapacityScheduler() {}
 
 
   @Override
   @Override
@@ -172,6 +175,21 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
     return maximumAllocation;
     return maximumAllocation;
   }
   }
 
 
+  @Override
+  public Comparator<FiCaSchedulerApp> getApplicationComparator() {
+    return applicationComparator;
+  }
+
+  @Override
+  public ResourceCalculator getResourceCalculator() {
+    return calculator;
+  }
+
+  @Override
+  public Comparator<CSQueue> getQueueComparator() {
+    return queueComparator;
+  }
+
   @Override
   @Override
   public synchronized int getNumClusterNodes() {
   public synchronized int getNumClusterNodes() {
     return numNodeManagers;
     return numNodeManagers;
@@ -192,11 +210,20 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
       reinitialize(Configuration conf, RMContext rmContext) throws IOException {
       reinitialize(Configuration conf, RMContext rmContext) throws IOException {
     if (!initialized) {
     if (!initialized) {
       this.conf = new CapacitySchedulerConfiguration(conf);
       this.conf = new CapacitySchedulerConfiguration(conf);
+      
       this.minimumAllocation = this.conf.getMinimumAllocation();
       this.minimumAllocation = this.conf.getMinimumAllocation();
       this.maximumAllocation = this.conf.getMaximumAllocation();
       this.maximumAllocation = this.conf.getMaximumAllocation();
+      this.calculator = this.conf.getResourceCalculator();
+
       this.rmContext = rmContext;
       this.rmContext = rmContext;
+      
       initializeQueues(this.conf);
       initializeQueues(this.conf);
+      
       initialized = true;
       initialized = true;
+      LOG.info("Initialized CapacityScheduler with " +
+          "calculator=" + getResourceCalculator().getClass() + ", " +
+          "minimumAllocation=<" + getMinimumResourceCapability() + ">, " +
+          "maximumAllocation=<" + getMaximumResourceCapability() + ">");
     } else {
     } else {
 
 
       CapacitySchedulerConfiguration oldConf = this.conf; 
       CapacitySchedulerConfiguration oldConf = this.conf; 
@@ -226,8 +253,8 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
   private void initializeQueues(CapacitySchedulerConfiguration conf)
   private void initializeQueues(CapacitySchedulerConfiguration conf)
     throws IOException {
     throws IOException {
     root = 
     root = 
-        parseQueue(this, conf, null, CapacitySchedulerConfiguration.ROOT, queues, queues, 
-            queueComparator, applicationComparator, noop);
+        parseQueue(this, conf, null, CapacitySchedulerConfiguration.ROOT, 
+            queues, queues, noop);
     LOG.info("Initialized root queue " + root);
     LOG.info("Initialized root queue " + root);
   }
   }
 
 
@@ -237,8 +264,8 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
     // Parse new queues
     // Parse new queues
     Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
     Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
     CSQueue newRoot = 
     CSQueue newRoot = 
-        parseQueue(this, conf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues, 
-            queueComparator, applicationComparator, noop);
+        parseQueue(this, conf, null, CapacitySchedulerConfiguration.ROOT, 
+            newQueues, queues, noop); 
     
     
     // Ensure all existing queues are still present
     // Ensure all existing queues are still present
     validateExistingQueues(queues, newQueues);
     validateExistingQueues(queues, newQueues);
@@ -291,8 +318,6 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
       CapacitySchedulerConfiguration conf, 
       CapacitySchedulerConfiguration conf, 
       CSQueue parent, String queueName, Map<String, CSQueue> queues,
       CSQueue parent, String queueName, Map<String, CSQueue> queues,
       Map<String, CSQueue> oldQueues, 
       Map<String, CSQueue> oldQueues, 
-      Comparator<CSQueue> queueComparator,
-      Comparator<FiCaSchedulerApp> applicationComparator,
       QueueHook hook) throws IOException {
       QueueHook hook) throws IOException {
     CSQueue queue;
     CSQueue queue;
     String[] childQueueNames = 
     String[] childQueueNames = 
@@ -303,15 +328,14 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
         throw new IllegalStateException(
         throw new IllegalStateException(
             "Queue configuration missing child queue names for " + queueName);
             "Queue configuration missing child queue names for " + queueName);
       }
       }
-      queue = new LeafQueue(csContext, queueName, parent, applicationComparator,
-                            oldQueues.get(queueName));
+      queue = 
+          new LeafQueue(csContext, queueName, parent,oldQueues.get(queueName));
       
       
       // Used only for unit tests
       // Used only for unit tests
       queue = hook.hook(queue);
       queue = hook.hook(queue);
     } else {
     } else {
       ParentQueue parentQueue = 
       ParentQueue parentQueue = 
-        new ParentQueue(csContext, queueName, queueComparator, parent,
-                        oldQueues.get(queueName));
+        new ParentQueue(csContext, queueName, parent,oldQueues.get(queueName));
 
 
       // Used only for unit tests
       // Used only for unit tests
       queue = hook.hook(parentQueue);
       queue = hook.hook(parentQueue);
@@ -320,7 +344,7 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
       for (String childQueueName : childQueueNames) {
       for (String childQueueName : childQueueNames) {
         CSQueue childQueue = 
         CSQueue childQueue = 
           parseQueue(csContext, conf, queue, childQueueName, 
           parseQueue(csContext, conf, queue, childQueueName, 
-              queues, oldQueues, queueComparator, applicationComparator, hook);
+              queues, oldQueues, hook);
         childQueues.add(childQueue);
         childQueues.add(childQueue);
       }
       }
       parentQueue.setChildQueues(childQueues);
       parentQueue.setChildQueues(childQueues);
@@ -442,7 +466,7 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
   }
   }
 
 
   private static final Allocation EMPTY_ALLOCATION = 
   private static final Allocation EMPTY_ALLOCATION = 
-      new Allocation(EMPTY_CONTAINER_LIST, Resources.createResource(0));
+      new Allocation(EMPTY_CONTAINER_LIST, Resources.createResource(0, 0));
 
 
   @Override
   @Override
   @Lock(Lock.NoLock.class)
   @Lock(Lock.NoLock.class)
@@ -457,7 +481,8 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
     }
     }
     
     
     // Sanity check
     // Sanity check
-    SchedulerUtils.normalizeRequests(ask, minimumAllocation.getMemory());
+    SchedulerUtils.normalizeRequests(
+        ask, calculator, getClusterResources(), minimumAllocation);
 
 
     // Release containers
     // Release containers
     for (ContainerId releasedContainerId : release) {
     for (ContainerId releasedContainerId : release) {

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

@@ -26,11 +26,14 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 
 
 public class CapacitySchedulerConfiguration extends Configuration {
 public class CapacitySchedulerConfiguration extends Configuration {
@@ -112,6 +115,13 @@ public class CapacitySchedulerConfiguration extends Configuration {
       PREFIX +"user-metrics.enable";
       PREFIX +"user-metrics.enable";
   @Private public static final boolean DEFAULT_ENABLE_USER_METRICS = false;
   @Private public static final boolean DEFAULT_ENABLE_USER_METRICS = false;
 
 
+  /** ResourceComparator for scheduling. */
+  @Private public static final String RESOURCE_CALCULATOR_CLASS =
+      PREFIX + "resource-calculator";
+
+  @Private public static final Class<? extends ResourceCalculator> 
+  DEFAULT_RESOURCE_CALCULATOR_CLASS = DefaultResourceCalculator.class;
+  
   @Private
   @Private
   public static final String ROOT = "root";
   public static final String ROOT = "root";
 
 
@@ -289,14 +299,20 @@ public class CapacitySchedulerConfiguration extends Configuration {
     int minimumMemory = getInt(
     int minimumMemory = getInt(
         YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
         YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
-    return Resources.createResource(minimumMemory);
+    int minimumCores = getInt(
+        YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+    return Resources.createResource(minimumMemory, minimumCores);
   }
   }
 
 
   public Resource getMaximumAllocation() {
   public Resource getMaximumAllocation() {
     int maximumMemory = getInt(
     int maximumMemory = getInt(
         YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
         YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
-    return Resources.createResource(maximumMemory);
+    int maximumCores = getInt(
+        YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_CORES,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_CORES);
+    return Resources.createResource(maximumMemory, maximumCores);
   }
   }
 
 
   public boolean getEnableUserMetrics() {
   public boolean getEnableUserMetrics() {
@@ -307,4 +323,21 @@ public class CapacitySchedulerConfiguration extends Configuration {
     int delay = getInt(NODE_LOCALITY_DELAY, DEFAULT_NODE_LOCALITY_DELAY);
     int delay = getInt(NODE_LOCALITY_DELAY, DEFAULT_NODE_LOCALITY_DELAY);
     return (delay == DEFAULT_NODE_LOCALITY_DELAY) ? 0 : delay;
     return (delay == DEFAULT_NODE_LOCALITY_DELAY) ? 0 : delay;
   }
   }
+  
+  public ResourceCalculator getResourceCalculator() {
+    return ReflectionUtils.newInstance(
+        getClass(
+            RESOURCE_CALCULATOR_CLASS, 
+            DEFAULT_RESOURCE_CALCULATOR_CLASS, 
+            ResourceCalculator.class), 
+        this);
+  }
+
+  public void setResourceComparator(
+      Class<? extends ResourceCalculator> resourceCalculatorClass) {
+    setClass(
+        RESOURCE_CALCULATOR_CLASS, 
+        resourceCalculatorClass, 
+        ResourceCalculator.class);
+  }
 }
 }

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

@@ -18,10 +18,14 @@
 
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 
+import java.util.Comparator;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 
 
 /**
 /**
  * Read-only interface to {@link CapacityScheduler} context.
  * Read-only interface to {@link CapacityScheduler} context.
@@ -45,4 +49,10 @@ public interface CapacitySchedulerContext {
    * Get the yarn configuration.
    * Get the yarn configuration.
    */
    */
   Configuration getConf();
   Configuration getConf();
+
+  Comparator<FiCaSchedulerApp> getApplicationComparator();
+
+  ResourceCalculator getResourceCalculator();
+
+  Comparator<CSQueue> getQueueComparator();
 }
 }

+ 107 - 70
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

@@ -53,6 +53,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
@@ -90,7 +91,7 @@ public class LeafQueue implements CSQueue {
   private int maxActiveAppsUsingAbsCap; // Based on absolute capacity
   private int maxActiveAppsUsingAbsCap; // Based on absolute capacity
   private int maxActiveApplicationsPerUser;
   private int maxActiveApplicationsPerUser;
   
   
-  private Resource usedResources = Resources.createResource(0);
+  private Resource usedResources = Resources.createResource(0, 0);
   private float usedCapacity = 0.0f;
   private float usedCapacity = 0.0f;
   private volatile int numContainers;
   private volatile int numContainers;
 
 
@@ -126,12 +127,16 @@ public class LeafQueue implements CSQueue {
   
   
   private final int nodeLocalityDelay;
   private final int nodeLocalityDelay;
   
   
+  private final ResourceCalculator resourceCalculator;
+  
   public LeafQueue(CapacitySchedulerContext cs, 
   public LeafQueue(CapacitySchedulerContext cs, 
-      String queueName, CSQueue parent, 
-      Comparator<FiCaSchedulerApp> applicationComparator, CSQueue old) {
+      String queueName, CSQueue parent, CSQueue old) {
     this.scheduler = cs;
     this.scheduler = cs;
     this.queueName = queueName;
     this.queueName = queueName;
     this.parent = parent;
     this.parent = parent;
+    
+    this.resourceCalculator = cs.getResourceCalculator();
+
     // must be after parent and queueName are initialized
     // must be after parent and queueName are initialized
     this.metrics = old != null ? old.getMetrics() :
     this.metrics = old != null ? old.getMetrics() :
         QueueMetrics.forQueue(getQueuePath(), parent,
         QueueMetrics.forQueue(getQueuePath(), parent,
@@ -141,8 +146,9 @@ public class LeafQueue implements CSQueue {
     this.minimumAllocation = cs.getMinimumResourceCapability();
     this.minimumAllocation = cs.getMinimumResourceCapability();
     this.maximumAllocation = cs.getMaximumResourceCapability();
     this.maximumAllocation = cs.getMaximumResourceCapability();
     this.minimumAllocationFactor = 
     this.minimumAllocationFactor = 
-        (float)(maximumAllocation.getMemory() - minimumAllocation.getMemory()) / 
-         maximumAllocation.getMemory();
+        Resources.ratio(resourceCalculator, 
+            Resources.subtract(maximumAllocation, minimumAllocation), 
+            maximumAllocation);
     this.containerTokenSecretManager = cs.getContainerTokenSecretManager();
     this.containerTokenSecretManager = cs.getContainerTokenSecretManager();
 
 
     float capacity = 
     float capacity = 
@@ -171,10 +177,12 @@ public class LeafQueue implements CSQueue {
             getMaximumApplicationMasterResourcePerQueuePercent(getQueuePath());
             getMaximumApplicationMasterResourcePerQueuePercent(getQueuePath());
     int maxActiveApplications = 
     int maxActiveApplications = 
         CSQueueUtils.computeMaxActiveApplications(
         CSQueueUtils.computeMaxActiveApplications(
+            resourceCalculator,
             cs.getClusterResources(), this.minimumAllocation,
             cs.getClusterResources(), this.minimumAllocation,
             maxAMResourcePerQueuePercent, absoluteMaxCapacity);
             maxAMResourcePerQueuePercent, absoluteMaxCapacity);
     this.maxActiveAppsUsingAbsCap = 
     this.maxActiveAppsUsingAbsCap = 
             CSQueueUtils.computeMaxActiveApplications(
             CSQueueUtils.computeMaxActiveApplications(
+                resourceCalculator,
                 cs.getClusterResources(), this.minimumAllocation,
                 cs.getClusterResources(), this.minimumAllocation,
                 maxAMResourcePerQueuePercent, absoluteCapacity);
                 maxAMResourcePerQueuePercent, absoluteCapacity);
     int maxActiveApplicationsPerUser = 
     int maxActiveApplicationsPerUser = 
@@ -207,6 +215,8 @@ public class LeafQueue implements CSQueue {
         + ", fullname=" + getQueuePath());
         + ", fullname=" + getQueuePath());
     }
     }
 
 
+    Comparator<FiCaSchedulerApp> applicationComparator =
+        cs.getApplicationComparator();
     this.pendingApplications = 
     this.pendingApplications = 
         new TreeSet<FiCaSchedulerApp>(applicationComparator);
         new TreeSet<FiCaSchedulerApp>(applicationComparator);
     this.activeApplications = new TreeSet<FiCaSchedulerApp>(applicationComparator);
     this.activeApplications = new TreeSet<FiCaSchedulerApp>(applicationComparator);
@@ -256,7 +266,8 @@ public class LeafQueue implements CSQueue {
     
     
     // Update metrics
     // Update metrics
     CSQueueUtils.updateQueueStatistics(
     CSQueueUtils.updateQueueStatistics(
-        this, getParent(), clusterResource, minimumAllocation);
+        resourceCalculator, this, getParent(), clusterResource, 
+        minimumAllocation);
 
 
     LOG.info("Initializing " + queueName + "\n" +
     LOG.info("Initializing " + queueName + "\n" +
         "capacity = " + capacity +
         "capacity = " + capacity +
@@ -545,7 +556,7 @@ public class LeafQueue implements CSQueue {
     return queueName + ": " + 
     return queueName + ": " + 
         "capacity=" + capacity + ", " + 
         "capacity=" + capacity + ", " + 
         "absoluteCapacity=" + absoluteCapacity + ", " + 
         "absoluteCapacity=" + absoluteCapacity + ", " + 
-        "usedResources=" + usedResources.getMemory() + "MB, " + 
+        "usedResources=" + usedResources +  
         "usedCapacity=" + getUsedCapacity() + ", " + 
         "usedCapacity=" + getUsedCapacity() + ", " + 
         "absoluteUsedCapacity=" + getAbsoluteUsedCapacity() + ", " +
         "absoluteUsedCapacity=" + getAbsoluteUsedCapacity() + ", " +
         "numApps=" + getNumApplications() + ", " + 
         "numApps=" + getNumApplications() + ", " + 
@@ -754,7 +765,7 @@ public class LeafQueue implements CSQueue {
   }
   }
 
 
   private static final CSAssignment NULL_ASSIGNMENT =
   private static final CSAssignment NULL_ASSIGNMENT =
-      new CSAssignment(Resources.createResource(0), NodeType.NODE_LOCAL);
+      new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
   
   
   @Override
   @Override
   public synchronized CSAssignment 
   public synchronized CSAssignment 
@@ -813,7 +824,8 @@ public class LeafQueue implements CSQueue {
           }
           }
 
 
           // Check user limit
           // Check user limit
-          if (!assignToUser(application.getUser(), userLimit)) {
+          if (!assignToUser(
+              clusterResource, application.getUser(), userLimit)) {
             break; 
             break; 
           }
           }
 
 
@@ -827,7 +839,8 @@ public class LeafQueue implements CSQueue {
 
 
           // Did we schedule or reserve a container?
           // Did we schedule or reserve a container?
           Resource assigned = assignment.getResource();
           Resource assigned = assignment.getResource();
-          if (Resources.greaterThan(assigned, Resources.none())) {
+          if (Resources.greaterThan(
+              resourceCalculator, clusterResource, assigned, Resources.none())) {
 
 
             // Book-keeping 
             // Book-keeping 
             // Note: Update headroom to account for current allocation too...
             // Note: Update headroom to account for current allocation too...
@@ -882,21 +895,25 @@ public class LeafQueue implements CSQueue {
     
     
     // Doesn't matter... since it's already charged for at time of reservation
     // Doesn't matter... since it's already charged for at time of reservation
     // "re-reservation" is *free*
     // "re-reservation" is *free*
-    return org.apache.hadoop.yarn.server.resourcemanager.resource.Resource.NONE;
+    return Resources.none();
   }
   }
 
 
   private synchronized boolean assignToQueue(Resource clusterResource, 
   private synchronized boolean assignToQueue(Resource clusterResource, 
       Resource required) {
       Resource required) {
     // Check how of the cluster's absolute capacity we are currently using...
     // Check how of the cluster's absolute capacity we are currently using...
     float potentialNewCapacity = 
     float potentialNewCapacity = 
-      (float)(usedResources.getMemory() + required.getMemory()) / 
-        clusterResource.getMemory();
+        Resources.divide(
+            resourceCalculator, clusterResource, 
+            Resources.add(usedResources, required), 
+            clusterResource);
     if (potentialNewCapacity > absoluteMaxCapacity) {
     if (potentialNewCapacity > absoluteMaxCapacity) {
       LOG.info(getQueueName() + 
       LOG.info(getQueueName() + 
-          " usedResources: " + usedResources.getMemory() +
-          " clusterResources: " + clusterResource.getMemory() +
-          " currentCapacity " + ((float)usedResources.getMemory())/clusterResource.getMemory() + 
-          " required " + required.getMemory() +
+          " usedResources: " + usedResources +
+          " clusterResources: " + clusterResource +
+          " currentCapacity " + 
+            Resources.divide(resourceCalculator, clusterResource, 
+                usedResources, clusterResource) + 
+          " required " + required +
           " potentialNewCapacity: " + potentialNewCapacity + " ( " +
           " potentialNewCapacity: " + potentialNewCapacity + " ( " +
           " max-capacity: " + absoluteMaxCapacity + ")");
           " max-capacity: " + absoluteMaxCapacity + ")");
       return false;
       return false;
@@ -919,14 +936,18 @@ public class LeafQueue implements CSQueue {
     
     
 
 
     Resource queueMaxCap =                        // Queue Max-Capacity
     Resource queueMaxCap =                        // Queue Max-Capacity
-        Resources.createResource(
-            CSQueueUtils.roundDown(minimumAllocation, 
-                (int)(absoluteMaxCapacity * clusterResource.getMemory()))
-            );
+        Resources.multiplyAndNormalizeDown(
+            resourceCalculator, 
+            clusterResource, 
+            absoluteMaxCapacity, 
+            minimumAllocation);
     
     
     Resource userConsumed = getUser(user).getConsumedResources(); 
     Resource userConsumed = getUser(user).getConsumedResources(); 
     Resource headroom = 
     Resource headroom = 
-        Resources.subtract(Resources.min(userLimit, queueMaxCap), userConsumed);
+        Resources.subtract(
+            Resources.min(resourceCalculator, clusterResource, 
+                userLimit, queueMaxCap), 
+            userConsumed);
     
     
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Headroom calculation for user " + user + ": " + 
       LOG.debug("Headroom calculation for user " + user + ": " + 
@@ -953,35 +974,46 @@ public class LeafQueue implements CSQueue {
     //   (usedResources + required) (which extra resources we are allocating)
     //   (usedResources + required) (which extra resources we are allocating)
 
 
     // Allow progress for queues with miniscule capacity
     // Allow progress for queues with miniscule capacity
-    final int queueCapacity = 
-      Math.max(
-          CSQueueUtils.roundUp(
-              minimumAllocation, 
-              (int)(absoluteCapacity * clusterResource.getMemory())), 
-          required.getMemory()
-          );
-
-    final int consumed = usedResources.getMemory();
-    final int currentCapacity = 
-      (consumed < queueCapacity) ? 
-          queueCapacity : (consumed + required.getMemory());
-
+    final Resource queueCapacity =
+        Resources.max(
+            resourceCalculator, clusterResource, 
+            Resources.multiplyAndNormalizeUp(
+                resourceCalculator, 
+                clusterResource, 
+                absoluteCapacity, 
+                minimumAllocation), 
+            required);
+
+    Resource currentCapacity =
+        Resources.lessThan(resourceCalculator, clusterResource, 
+            usedResources, queueCapacity) ?
+            queueCapacity : Resources.add(usedResources, required);
+    
     // Never allow a single user to take more than the 
     // Never allow a single user to take more than the 
     // queue's configured capacity * user-limit-factor.
     // queue's configured capacity * user-limit-factor.
     // Also, the queue's configured capacity should be higher than 
     // Also, the queue's configured capacity should be higher than 
     // queue-hard-limit * ulMin
     // queue-hard-limit * ulMin
     
     
     final int activeUsers = activeUsersManager.getNumActiveUsers();  
     final int activeUsers = activeUsersManager.getNumActiveUsers();  
-
-    int limit = 
-      CSQueueUtils.roundUp(
-          minimumAllocation,
-          Math.min(
-              Math.max(divideAndCeil(currentCapacity, activeUsers), 
-                       divideAndCeil((int)userLimit*currentCapacity, 100)),
-              (int)(queueCapacity * userLimitFactor)
-              )
-          );
+    		
+    Resource limit =
+        Resources.roundUp(
+            resourceCalculator, 
+            Resources.min(
+                resourceCalculator, clusterResource,   
+                Resources.max(
+                    resourceCalculator, clusterResource, 
+                    Resources.divideAndCeil(
+                        resourceCalculator, currentCapacity, activeUsers),
+                    Resources.divideAndCeil(
+                        resourceCalculator, 
+                        Resources.multiplyAndRoundDown(
+                            currentCapacity, userLimit), 
+                        100)
+                    ), 
+                Resources.multiplyAndRoundDown(queueCapacity, userLimitFactor)
+                ), 
+            minimumAllocation);
 
 
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       String userName = application.getUser();
       String userName = application.getUser();
@@ -993,23 +1025,25 @@ public class LeafQueue implements CSQueue {
           " consumed: " + getUser(userName).getConsumedResources() + 
           " consumed: " + getUser(userName).getConsumedResources() + 
           " limit: " + limit +
           " limit: " + limit +
           " queueCapacity: " + queueCapacity + 
           " queueCapacity: " + queueCapacity + 
-          " qconsumed: " + consumed +
+          " qconsumed: " + usedResources +
           " currentCapacity: " + currentCapacity +
           " currentCapacity: " + currentCapacity +
           " activeUsers: " + activeUsers +
           " activeUsers: " + activeUsers +
-          " clusterCapacity: " + clusterResource.getMemory()
+          " clusterCapacity: " + clusterResource
       );
       );
     }
     }
 
 
-    return Resources.createResource(limit);
+    return limit;
   }
   }
   
   
-  private synchronized boolean assignToUser(String userName, Resource limit) {
+  private synchronized boolean assignToUser(Resource clusterResource,
+      String userName, Resource limit) {
 
 
     User user = getUser(userName);
     User user = getUser(userName);
     
     
     // Note: We aren't considering the current request since there is a fixed
     // Note: We aren't considering the current request since there is a fixed
-    // overhead of the AM, but it's a > check, not a >= check, so... 
-    if ((user.getConsumedResources().getMemory()) > limit.getMemory()) {
+    // overhead of the AM, but it's a > check, not a >= check, so...
+    if (Resources.greaterThan(resourceCalculator, clusterResource, 
+            user.getConsumedResources(), limit)) {
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("User " + userName + " in queue " + getQueueName() + 
         LOG.debug("User " + userName + " in queue " + getQueueName() + 
             " will exceed limit - " +  
             " will exceed limit - " +  
@@ -1023,21 +1057,15 @@ public class LeafQueue implements CSQueue {
     return true;
     return true;
   }
   }
 
 
-  static int divideAndCeil(int a, int b) {
-    if (b == 0) {
-      LOG.info("divideAndCeil called with a=" + a + " b=" + b);
-      return 0;
-    }
-    return (a + (b - 1)) / b;
-  }
-
   boolean needContainers(FiCaSchedulerApp application, Priority priority, Resource required) {
   boolean needContainers(FiCaSchedulerApp application, Priority priority, Resource required) {
     int requiredContainers = application.getTotalRequiredResources(priority);
     int requiredContainers = application.getTotalRequiredResources(priority);
     int reservedContainers = application.getNumReservedContainers(priority);
     int reservedContainers = application.getNumReservedContainers(priority);
     int starvation = 0;
     int starvation = 0;
     if (reservedContainers > 0) {
     if (reservedContainers > 0) {
       float nodeFactor = 
       float nodeFactor = 
-          ((float)required.getMemory() / getMaximumAllocation().getMemory());
+          Resources.ratio(
+              resourceCalculator, required, getMaximumAllocation()
+              );
       
       
       // Use percentage of node required to bias against large containers...
       // Use percentage of node required to bias against large containers...
       // Protect against corner case where you need the whole node with
       // Protect against corner case where you need the whole node with
@@ -1052,7 +1080,7 @@ public class LeafQueue implements CSQueue {
             " app.#re-reserve=" + application.getReReservations(priority) + 
             " app.#re-reserve=" + application.getReReservations(priority) + 
             " reserved=" + reservedContainers + 
             " reserved=" + reservedContainers + 
             " nodeFactor=" + nodeFactor + 
             " nodeFactor=" + nodeFactor + 
-            " minAllocFactor=" + minimumAllocationFactor +
+            " minAllocFactor=" + getMinimumAllocationFactor() +
             " starvation=" + starvation);
             " starvation=" + starvation);
       }
       }
     }
     }
@@ -1069,7 +1097,8 @@ public class LeafQueue implements CSQueue {
     assigned = 
     assigned = 
         assignNodeLocalContainers(clusterResource, node, application, priority,
         assignNodeLocalContainers(clusterResource, node, application, priority,
             reservedContainer); 
             reservedContainer); 
-    if (Resources.greaterThan(assigned, Resources.none())) {
+    if (Resources.greaterThan(resourceCalculator, clusterResource, 
+            assigned, Resources.none())) {
       return new CSAssignment(assigned, NodeType.NODE_LOCAL);
       return new CSAssignment(assigned, NodeType.NODE_LOCAL);
     }
     }
 
 
@@ -1077,7 +1106,8 @@ public class LeafQueue implements CSQueue {
     assigned = 
     assigned = 
         assignRackLocalContainers(clusterResource, node, application, priority, 
         assignRackLocalContainers(clusterResource, node, application, priority, 
             reservedContainer);
             reservedContainer);
-    if (Resources.greaterThan(assigned, Resources.none())) {
+    if (Resources.greaterThan(resourceCalculator, clusterResource, 
+            assigned, Resources.none())) {
       return new CSAssignment(assigned, NodeType.RACK_LOCAL);
       return new CSAssignment(assigned, NodeType.RACK_LOCAL);
     }
     }
     
     
@@ -1231,7 +1261,8 @@ public class LeafQueue implements CSQueue {
 
 
     Resource available = node.getAvailableResource();
     Resource available = node.getAvailableResource();
 
 
-    assert (available.getMemory() >  0);
+    assert Resources.greaterThan(
+        resourceCalculator, clusterResource, available, Resources.none());
 
 
     // Create the container if necessary
     // Create the container if necessary
     Container container = 
     Container container = 
@@ -1239,12 +1270,13 @@ public class LeafQueue implements CSQueue {
   
   
     // something went wrong getting/creating the container 
     // something went wrong getting/creating the container 
     if (container == null) {
     if (container == null) {
+      LOG.warn("Couldn't get container for allocation!");
       return Resources.none();
       return Resources.none();
     }
     }
 
 
     // Can we allocate a container on this node?
     // Can we allocate a container on this node?
     int availableContainers = 
     int availableContainers = 
-        available.getMemory() / capability.getMemory();         
+        resourceCalculator.computeAvailableContainers(available, capability);
     if (availableContainers > 0) {
     if (availableContainers > 0) {
       // Allocate...
       // Allocate...
 
 
@@ -1267,8 +1299,9 @@ public class LeafQueue implements CSQueue {
       // Inform the application
       // Inform the application
       RMContainer allocatedContainer = 
       RMContainer allocatedContainer = 
           application.allocate(type, node, priority, request, container);
           application.allocate(type, node, priority, request, container);
+
+      // Does the application need this resource?
       if (allocatedContainer == null) {
       if (allocatedContainer == null) {
-        // Did the application need this resource?
         return Resources.none();
         return Resources.none();
       }
       }
 
 
@@ -1379,7 +1412,7 @@ public class LeafQueue implements CSQueue {
     // Update queue metrics
     // Update queue metrics
     Resources.addTo(usedResources, resource);
     Resources.addTo(usedResources, resource);
     CSQueueUtils.updateQueueStatistics(
     CSQueueUtils.updateQueueStatistics(
-        this, getParent(), clusterResource, minimumAllocation);
+        resourceCalculator, this, getParent(), clusterResource, minimumAllocation);
     ++numContainers;
     ++numContainers;
 
 
     // Update user metrics
     // Update user metrics
@@ -1404,7 +1437,8 @@ public class LeafQueue implements CSQueue {
     // Update queue metrics
     // Update queue metrics
     Resources.subtractFrom(usedResources, resource);
     Resources.subtractFrom(usedResources, resource);
     CSQueueUtils.updateQueueStatistics(
     CSQueueUtils.updateQueueStatistics(
-        this, getParent(), clusterResource, minimumAllocation);
+        resourceCalculator, this, getParent(), clusterResource, 
+        minimumAllocation);
     --numContainers;
     --numContainers;
 
 
     // Update user metrics
     // Update user metrics
@@ -1423,10 +1457,12 @@ public class LeafQueue implements CSQueue {
     // Update queue properties
     // Update queue properties
     maxActiveApplications = 
     maxActiveApplications = 
         CSQueueUtils.computeMaxActiveApplications(
         CSQueueUtils.computeMaxActiveApplications(
+            resourceCalculator,
             clusterResource, minimumAllocation, 
             clusterResource, minimumAllocation, 
             maxAMResourcePerQueuePercent, absoluteMaxCapacity);
             maxAMResourcePerQueuePercent, absoluteMaxCapacity);
     maxActiveAppsUsingAbsCap = 
     maxActiveAppsUsingAbsCap = 
         CSQueueUtils.computeMaxActiveApplications(
         CSQueueUtils.computeMaxActiveApplications(
+            resourceCalculator,
             clusterResource, minimumAllocation, 
             clusterResource, minimumAllocation, 
             maxAMResourcePerQueuePercent, absoluteCapacity);
             maxAMResourcePerQueuePercent, absoluteCapacity);
     maxActiveApplicationsPerUser = 
     maxActiveApplicationsPerUser = 
@@ -1435,7 +1471,8 @@ public class LeafQueue implements CSQueue {
     
     
     // Update metrics
     // Update metrics
     CSQueueUtils.updateQueueStatistics(
     CSQueueUtils.updateQueueStatistics(
-        this, getParent(), clusterResource, minimumAllocation);
+        resourceCalculator, this, getParent(), clusterResource, 
+        minimumAllocation);
     
     
     // Update application properties
     // Update application properties
     for (FiCaSchedulerApp application : activeApplications) {
     for (FiCaSchedulerApp application : activeApplications) {
@@ -1452,7 +1489,7 @@ public class LeafQueue implements CSQueue {
   }
   }
 
 
   static class User {
   static class User {
-    Resource consumed = Resources.createResource(0);
+    Resource consumed = Resources.createResource(0, 0);
     int pendingApplications = 0;
     int pendingApplications = 0;
     int activeApplications = 0;
     int activeApplications = 0;
 
 

+ 31 - 23
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

@@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
@@ -74,8 +75,7 @@ public class ParentQueue implements CSQueue {
   private final Set<CSQueue> childQueues;
   private final Set<CSQueue> childQueues;
   private final Comparator<CSQueue> queueComparator;
   private final Comparator<CSQueue> queueComparator;
   
   
-  private Resource usedResources = 
-    Resources.createResource(0);
+  private Resource usedResources = Resources.createResource(0, 0);
   
   
   private final boolean rootQueue;
   private final boolean rootQueue;
   
   
@@ -96,14 +96,16 @@ public class ParentQueue implements CSQueue {
   private final RecordFactory recordFactory = 
   private final RecordFactory recordFactory = 
     RecordFactoryProvider.getRecordFactory(null);
     RecordFactoryProvider.getRecordFactory(null);
 
 
+  private final ResourceCalculator resourceCalculator;
+  
   public ParentQueue(CapacitySchedulerContext cs, 
   public ParentQueue(CapacitySchedulerContext cs, 
-      String queueName, Comparator<CSQueue> comparator, 
-      CSQueue parent, CSQueue old) {
+      String queueName, CSQueue parent, CSQueue old) {
     minimumAllocation = cs.getMinimumResourceCapability();
     minimumAllocation = cs.getMinimumResourceCapability();
     
     
     this.parent = parent;
     this.parent = parent;
     this.queueName = queueName;
     this.queueName = queueName;
     this.rootQueue = (parent == null);
     this.rootQueue = (parent == null);
+    this.resourceCalculator = cs.getResourceCalculator();
 
 
     // must be called after parent and queueName is set
     // must be called after parent and queueName is set
     this.metrics = old != null ? old.getMetrics() :
     this.metrics = old != null ? old.getMetrics() :
@@ -143,7 +145,7 @@ public class ParentQueue implements CSQueue {
         capacity, absoluteCapacity, 
         capacity, absoluteCapacity, 
         maximumCapacity, absoluteMaxCapacity, state, acls);
         maximumCapacity, absoluteMaxCapacity, state, acls);
     
     
-    this.queueComparator = comparator;
+    this.queueComparator = cs.getQueueComparator();
     this.childQueues = new TreeSet<CSQueue>(queueComparator);
     this.childQueues = new TreeSet<CSQueue>(queueComparator);
 
 
     LOG.info("Initialized parent-queue " + queueName + 
     LOG.info("Initialized parent-queue " + queueName + 
@@ -182,7 +184,7 @@ public class ParentQueue implements CSQueue {
 
 
     // Update metrics
     // Update metrics
     CSQueueUtils.updateQueueStatistics(
     CSQueueUtils.updateQueueStatistics(
-        this, parent, clusterResource, minimumAllocation);
+        resourceCalculator, this, parent, clusterResource, minimumAllocation);
 
 
     LOG.info(queueName +
     LOG.info(queueName +
         ", capacity=" + capacity +
         ", capacity=" + capacity +
@@ -357,7 +359,7 @@ public class ParentQueue implements CSQueue {
         "numChildQueue= " + childQueues.size() + ", " + 
         "numChildQueue= " + childQueues.size() + ", " + 
         "capacity=" + capacity + ", " +  
         "capacity=" + capacity + ", " +  
         "absoluteCapacity=" + absoluteCapacity + ", " +
         "absoluteCapacity=" + absoluteCapacity + ", " +
-        "usedResources=" + usedResources.getMemory() + "MB, " + 
+        "usedResources=" + usedResources + 
         "usedCapacity=" + getUsedCapacity() + ", " + 
         "usedCapacity=" + getUsedCapacity() + ", " + 
         "numApps=" + getNumApplications() + ", " + 
         "numApps=" + getNumApplications() + ", " + 
         "numContainers=" + getNumContainers();
         "numContainers=" + getNumContainers();
@@ -540,9 +542,9 @@ public class ParentQueue implements CSQueue {
   public synchronized CSAssignment assignContainers(
   public synchronized CSAssignment assignContainers(
       Resource clusterResource, FiCaSchedulerNode node) {
       Resource clusterResource, FiCaSchedulerNode node) {
     CSAssignment assignment = 
     CSAssignment assignment = 
-        new CSAssignment(Resources.createResource(0), NodeType.NODE_LOCAL);
+        new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
     
     
-    while (canAssign(node)) {
+    while (canAssign(clusterResource, node)) {
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Trying to assign containers to child-queue of "
         LOG.debug("Trying to assign containers to child-queue of "
           + getQueueName());
           + getQueueName());
@@ -559,8 +561,9 @@ public class ParentQueue implements CSQueue {
       assignment.setType(assignedToChild.getType());
       assignment.setType(assignedToChild.getType());
       
       
       // Done if no child-queue assigned anything
       // Done if no child-queue assigned anything
-      if (Resources.greaterThan(assignedToChild.getResource(), 
-              Resources.none())) {
+      if (Resources.greaterThan(
+              resourceCalculator, clusterResource, 
+              assignedToChild.getResource(), Resources.none())) {
         // Track resource utilization for the parent-queue
         // Track resource utilization for the parent-queue
         allocateResource(clusterResource, assignedToChild.getResource());
         allocateResource(clusterResource, assignedToChild.getResource());
         
         
@@ -603,11 +606,14 @@ public class ParentQueue implements CSQueue {
 
 
   private synchronized boolean assignToQueue(Resource clusterResource) {
   private synchronized boolean assignToQueue(Resource clusterResource) {
     // Check how of the cluster's absolute capacity we are currently using...
     // Check how of the cluster's absolute capacity we are currently using...
-    float currentCapacity = 
-      (float)(usedResources.getMemory()) / clusterResource.getMemory();
+    float currentCapacity =
+        Resources.divide(
+            resourceCalculator, clusterResource, 
+            usedResources, clusterResource);
+    
     if (currentCapacity >= absoluteMaxCapacity) {
     if (currentCapacity >= absoluteMaxCapacity) {
       LOG.info(getQueueName() + 
       LOG.info(getQueueName() + 
-          " used=" + usedResources.getMemory() + 
+          " used=" + usedResources + 
           " current-capacity (" + currentCapacity + ") " +
           " current-capacity (" + currentCapacity + ") " +
           " >= max-capacity (" + absoluteMaxCapacity + ")");
           " >= max-capacity (" + absoluteMaxCapacity + ")");
       return false;
       return false;
@@ -616,16 +622,16 @@ public class ParentQueue implements CSQueue {
 
 
   }
   }
   
   
-  private boolean canAssign(FiCaSchedulerNode node) {
+  private boolean canAssign(Resource clusterResource, FiCaSchedulerNode node) {
     return (node.getReservedContainer() == null) && 
     return (node.getReservedContainer() == null) && 
-        Resources.greaterThanOrEqual(node.getAvailableResource(), 
-                                     minimumAllocation);
+        Resources.greaterThanOrEqual(resourceCalculator, clusterResource, 
+            node.getAvailableResource(), minimumAllocation);
   }
   }
   
   
   synchronized CSAssignment assignContainersToChildQueues(Resource cluster, 
   synchronized CSAssignment assignContainersToChildQueues(Resource cluster, 
       FiCaSchedulerNode node) {
       FiCaSchedulerNode node) {
     CSAssignment assignment = 
     CSAssignment assignment = 
-        new CSAssignment(Resources.createResource(0), NodeType.NODE_LOCAL);
+        new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
     
     
     printChildQueues();
     printChildQueues();
 
 
@@ -640,11 +646,13 @@ public class ParentQueue implements CSQueue {
       if(LOG.isDebugEnabled()) {
       if(LOG.isDebugEnabled()) {
         LOG.debug("Assigned to queue: " + childQueue.getQueuePath() +
         LOG.debug("Assigned to queue: " + childQueue.getQueuePath() +
           " stats: " + childQueue + " --> " + 
           " stats: " + childQueue + " --> " + 
-          assignment.getResource().getMemory() + ", " + assignment.getType());
+          assignment.getResource() + ", " + assignment.getType());
       }
       }
 
 
       // If we do assign, remove the queue and re-insert in-order to re-sort
       // If we do assign, remove the queue and re-insert in-order to re-sort
-      if (Resources.greaterThan(assignment.getResource(), Resources.none())) {
+      if (Resources.greaterThan(
+              resourceCalculator, cluster, 
+              assignment.getResource(), Resources.none())) {
         // Remove and re-insert to sort
         // Remove and re-insert to sort
         iter.remove();
         iter.remove();
         LOG.info("Re-sorting queues since queue: " + childQueue.getQueuePath() + 
         LOG.info("Re-sorting queues since queue: " + childQueue.getQueuePath() + 
@@ -705,7 +713,7 @@ public class ParentQueue implements CSQueue {
       Resource resource) {
       Resource resource) {
     Resources.addTo(usedResources, resource);
     Resources.addTo(usedResources, resource);
     CSQueueUtils.updateQueueStatistics(
     CSQueueUtils.updateQueueStatistics(
-        this, parent, clusterResource, minimumAllocation);
+        resourceCalculator, this, parent, clusterResource, minimumAllocation);
     ++numContainers;
     ++numContainers;
   }
   }
   
   
@@ -713,7 +721,7 @@ public class ParentQueue implements CSQueue {
       Resource resource) {
       Resource resource) {
     Resources.subtractFrom(usedResources, resource);
     Resources.subtractFrom(usedResources, resource);
     CSQueueUtils.updateQueueStatistics(
     CSQueueUtils.updateQueueStatistics(
-        this, parent, clusterResource, minimumAllocation);
+        resourceCalculator, this, parent, clusterResource, minimumAllocation);
     --numContainers;
     --numContainers;
   }
   }
 
 
@@ -726,7 +734,7 @@ public class ParentQueue implements CSQueue {
     
     
     // Update metrics
     // Update metrics
     CSQueueUtils.updateQueueStatistics(
     CSQueueUtils.updateQueueStatistics(
-        this, parent, clusterResource, minimumAllocation);
+        resourceCalculator, this, parent, clusterResource, minimumAllocation);
   }
   }
   
   
   @Override
   @Override

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java

@@ -65,6 +65,7 @@ public class FiCaSchedulerNode extends SchedulerNode {
   public FiCaSchedulerNode(RMNode node) {
   public FiCaSchedulerNode(RMNode node) {
     this.rmNode = node;
     this.rmNode = node;
     this.availableResource.setMemory(node.getTotalCapability().getMemory());
     this.availableResource.setMemory(node.getTotalCapability().getMemory());
+    this.availableResource.setVirtualCores(node.getTotalCapability().getVirtualCores());
   }
   }
 
 
   public RMNode getRMNode() {
   public RMNode getRMNode() {

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java

@@ -33,7 +33,6 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 
 
 public class FSLeafQueue extends FSQueue {
 public class FSLeafQueue extends FSQueue {
   private static final Log LOG = LogFactory.getLog(
   private static final Log LOG = LogFactory.getLog(

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java

@@ -28,7 +28,6 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 
 
 public class FSParentQueue extends FSQueue {
 public class FSParentQueue extends FSQueue {
   private static final Log LOG = LogFactory.getLog(
   private static final Log LOG = LogFactory.getLog(

+ 28 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -53,7 +53,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@@ -637,6 +636,33 @@ public class FairScheduler implements ResourceScheduler {
         " cluster capacity: " + clusterCapacity);
         " cluster capacity: " + clusterCapacity);
   }
   }
 
 
+  /**
+   * Utility method to normalize a list of resource requests, by ensuring that
+   * the memory for each request is a multiple of minMemory and is not zero.
+   *
+   * @param asks a list of resource requests
+   * @param minMemory the configured minimum memory allocation
+   */
+  static void normalizeRequests(List<ResourceRequest> asks,
+      int minMemory) {
+    for (ResourceRequest ask : asks) {
+      normalizeRequest(ask, minMemory);
+    }
+  }
+
+  /**
+   * Utility method to normalize a resource request, by ensuring that the
+   * requested memory is a multiple of minMemory and is not zero.
+   *
+   * @param ask the resource request
+   * @param minMemory the configured minimum memory allocation
+   */
+  static void normalizeRequest(ResourceRequest ask, int minMemory) {
+    int memory = Math.max(ask.getCapability().getMemory(), minMemory);
+    ask.getCapability().setMemory(
+        minMemory * ((memory / minMemory) + (memory % minMemory > 0 ? 1 : 0)));
+  }
+
   @Override
   @Override
   public Allocation allocate(ApplicationAttemptId appAttemptId,
   public Allocation allocate(ApplicationAttemptId appAttemptId,
       List<ResourceRequest> ask, List<ContainerId> release) {
       List<ResourceRequest> ask, List<ContainerId> release) {
@@ -650,7 +676,7 @@ public class FairScheduler implements ResourceScheduler {
     }
     }
 
 
     // Sanity check
     // Sanity check
-    SchedulerUtils.normalizeRequests(ask, minimumAllocation.getMemory());
+    normalizeRequests(ask, minimumAllocation.getMemory());
 
 
     // Release containers
     // Release containers
     for (ContainerId releasedContainerId : release) {
     for (ContainerId releasedContainerId : release) {

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java

@@ -42,7 +42,6 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.w3c.dom.Document;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
 import org.w3c.dom.Node;

+ 150 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Resources.java

@@ -0,0 +1,150 @@
+/**
+* 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.scheduler.fair;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.Records;
+
+@Private
+@Evolving
+public class Resources {
+  
+  // Java doesn't have const :(
+  private static final Resource NONE = new Resource() {
+
+    @Override
+    public int getMemory() {
+      return 0;
+    }
+
+    @Override
+    public void setMemory(int memory) {
+      throw new RuntimeException("NONE cannot be modified!");
+    }
+
+    @Override
+    public int getVirtualCores() {
+      return 0;
+    }
+
+    @Override
+    public void setVirtualCores(int cores) {
+      throw new RuntimeException("NONE cannot be modified!");
+    }
+
+    @Override
+    public int compareTo(Resource o) {
+      int diff = 0 - o.getMemory();
+      if (diff == 0) {
+        diff = 0 - o.getVirtualCores();
+      }
+      return diff;
+    }
+    
+  };
+
+  public static Resource createResource(int memory) {
+    return createResource(memory, (memory > 0) ? 1 : 0);
+  }
+
+  public static Resource createResource(int memory, int cores) {
+    Resource resource = Records.newRecord(Resource.class);
+    resource.setMemory(memory);
+    resource.setVirtualCores(cores);
+    return resource;
+  }
+
+  public static Resource none() {
+    return NONE;
+  }
+
+  public static Resource clone(Resource res) {
+    return createResource(res.getMemory(), res.getVirtualCores());
+  }
+
+  public static Resource addTo(Resource lhs, Resource rhs) {
+    lhs.setMemory(lhs.getMemory() + rhs.getMemory());
+    return lhs;
+  }
+
+  public static Resource add(Resource lhs, Resource rhs) {
+    return addTo(clone(lhs), rhs);
+  }
+
+  public static Resource subtractFrom(Resource lhs, Resource rhs) {
+    lhs.setMemory(lhs.getMemory() - rhs.getMemory());
+    return lhs;
+  }
+
+  public static Resource subtract(Resource lhs, Resource rhs) {
+    return subtractFrom(clone(lhs), rhs);
+  }
+
+  public static Resource negate(Resource resource) {
+    return subtract(NONE, resource);
+  }
+
+  public static Resource multiplyTo(Resource lhs, int by) {
+    lhs.setMemory(lhs.getMemory() * by);
+    return lhs;
+  }
+
+  public static Resource multiply(Resource lhs, int by) {
+    return multiplyTo(clone(lhs), by);
+  }
+  
+  /**
+   * Mutliply a resource by a {@code double}. Note that integral 
+   * resource quantites are subject to rounding during cast.
+   */
+  public static Resource multiply(Resource lhs, double by) {
+    Resource out = clone(lhs);
+    out.setMemory((int) (lhs.getMemory() * by));
+    return out;
+  }
+
+  public static boolean equals(Resource lhs, Resource rhs) {
+    return lhs.getMemory() == rhs.getMemory();
+  }
+
+  public static boolean lessThan(Resource lhs, Resource rhs) {
+    return lhs.getMemory() < rhs.getMemory();
+  }
+
+  public static boolean lessThanOrEqual(Resource lhs, Resource rhs) {
+    return lhs.getMemory() <= rhs.getMemory();
+  }
+
+  public static boolean greaterThan(Resource lhs, Resource rhs) {
+    return lhs.getMemory() > rhs.getMemory();
+  }
+
+  public static boolean greaterThanOrEqual(Resource lhs, Resource rhs) {
+    return lhs.getMemory() >= rhs.getMemory();
+  }
+  
+  public static Resource min(Resource lhs, Resource rhs) {
+    return (lhs.getMemory() < rhs.getMemory()) ? lhs : rhs;
+  }
+
+  public static Resource max(Resource lhs, Resource rhs) {
+    return (lhs.getMemory() > rhs.getMemory()) ? lhs : rhs;
+  }}

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingAlgorithms.java

@@ -27,7 +27,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 
 
 /**
 /**
  * Utility class containing scheduling algorithms used in the fair scheduler.
  * Utility class containing scheduling algorithms used in the fair scheduler.

+ 10 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -57,6 +57,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
@@ -114,6 +116,8 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
 
 
   private static final String DEFAULT_QUEUE_NAME = "default";
   private static final String DEFAULT_QUEUE_NAME = "default";
   private QueueMetrics metrics;
   private QueueMetrics metrics;
+  
+  private final ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
 
 
   private final Queue DEFAULT_QUEUE = new Queue() {
   private final Queue DEFAULT_QUEUE = new Queue() {
     @Override
     @Override
@@ -226,7 +230,8 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
     }
     }
 
 
     // Sanity check
     // Sanity check
-    SchedulerUtils.normalizeRequests(ask, minimumAllocation.getMemory());
+    SchedulerUtils.normalizeRequests(ask, resourceCalculator, 
+        clusterResource, minimumAllocation);
 
 
     // Release containers
     // Release containers
     for (ContainerId releasedContainer : release) {
     for (ContainerId releasedContainer : release) {
@@ -371,7 +376,8 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
       application.showRequests();
       application.showRequests();
 
 
       // Done
       // Done
-      if (Resources.lessThan(node.getAvailableResource(), minimumAllocation)) {
+      if (Resources.lessThan(resourceCalculator, clusterResource,
+              node.getAvailableResource(), minimumAllocation)) {
         break;
         break;
       }
       }
     }
     }
@@ -588,8 +594,8 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
           completedContainer, RMContainerEventType.FINISHED);
           completedContainer, RMContainerEventType.FINISHED);
     }
     }
 
 
-    if (Resources.greaterThanOrEqual(node.getAvailableResource(),
-        minimumAllocation)) {
+    if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource,
+            node.getAvailableResource(),minimumAllocation)) {
       LOG.debug("Node heartbeat " + rmNode.getNodeID() + 
       LOG.debug("Node heartbeat " + rmNode.getNodeID() + 
           " available resource = " + node.getAvailableResource());
           " available resource = " + node.getAvailableResource());
 
 

+ 5 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java

@@ -77,14 +77,14 @@ public class NodeManager implements ContainerManager {
     new HashMap<ApplicationId, List<Container>>();
     new HashMap<ApplicationId, List<Container>>();
   
   
   public NodeManager(String hostName, int containerManagerPort, int httpPort,
   public NodeManager(String hostName, int containerManagerPort, int httpPort,
-      String rackName, int memory,
+      String rackName, Resource capability,
       ResourceTrackerService resourceTrackerService, RMContext rmContext)
       ResourceTrackerService resourceTrackerService, RMContext rmContext)
       throws IOException {
       throws IOException {
     this.containerManagerAddress = hostName + ":" + containerManagerPort;
     this.containerManagerAddress = hostName + ":" + containerManagerPort;
     this.nodeHttpAddress = hostName + ":" + httpPort;
     this.nodeHttpAddress = hostName + ":" + httpPort;
     this.rackName = rackName;
     this.rackName = rackName;
     this.resourceTrackerService = resourceTrackerService;
     this.resourceTrackerService = resourceTrackerService;
-    this.capability = Resources.createResource(memory);
+    this.capability = capability;
     Resources.addTo(available, capability);
     Resources.addTo(available, capability);
 
 
     this.nodeId = recordFactory.newRecordInstance(NodeId.class);
     this.nodeId = recordFactory.newRecordInstance(NodeId.class);
@@ -102,8 +102,10 @@ public class NodeManager implements ContainerManager {
         this.nodeId));
         this.nodeId));
    
    
     // Sanity check
     // Sanity check
-    Assert.assertEquals(memory, 
+    Assert.assertEquals(capability.getMemory(), 
        schedulerNode.getAvailableResource().getMemory());
        schedulerNode.getAvailableResource().getMemory());
+    Assert.assertEquals(capability.getVirtualCores(), 
+        schedulerNode.getAvailableResource().getVirtualCores());
   }
   }
   
   
   public String getHostName() {
   public String getHostName() {

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

@@ -168,7 +168,7 @@ public class TestApplicationACLs {
 
 
     ContainerLaunchContext amContainer = recordFactory
     ContainerLaunchContext amContainer = recordFactory
         .newRecordInstance(ContainerLaunchContext.class);
         .newRecordInstance(ContainerLaunchContext.class);
-    Resource resource = BuilderUtils.newResource(1024);
+    Resource resource = BuilderUtils.newResource(1024, 1);
     amContainer.setResource(resource);
     amContainer.setResource(resource);
     amContainer.setApplicationACLs(acls);
     amContainer.setApplicationACLs(acls);
     context.setAMContainerSpec(amContainer);
     context.setAMContainerSpec(amContainer);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java

@@ -304,9 +304,9 @@ public class TestClientRMTokens {
 
 
   private static ResourceScheduler createMockScheduler(Configuration conf) {
   private static ResourceScheduler createMockScheduler(Configuration conf) {
     ResourceScheduler mockSched = mock(ResourceScheduler.class);
     ResourceScheduler mockSched = mock(ResourceScheduler.class);
-    doReturn(BuilderUtils.newResource(512)).when(mockSched)
+    doReturn(BuilderUtils.newResource(512, 0)).when(mockSched)
         .getMinimumResourceCapability();
         .getMinimumResourceCapability();
-    doReturn(BuilderUtils.newResource(5120)).when(mockSched)
+    doReturn(BuilderUtils.newResource(5120, 0)).when(mockSched)
         .getMaximumResourceCapability();
         .getMaximumResourceCapability();
     return mockSched;
     return mockSched;
   }
   }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java

@@ -265,13 +265,13 @@ public class TestFifoScheduler {
     // Ask for a 1 GB container for app 1
     // Ask for a 1 GB container for app 1
     List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
     List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
     ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0), "*",
     ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0), "*",
-        BuilderUtils.newResource(GB), 1));
+        BuilderUtils.newResource(GB, 1), 1));
     fs.allocate(appAttemptId1, ask1, emptyId);
     fs.allocate(appAttemptId1, ask1, emptyId);
 
 
     // Ask for a 2 GB container for app 2
     // Ask for a 2 GB container for app 2
     List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
     List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
     ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0), "*",
     ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0), "*",
-        BuilderUtils.newResource(2 * GB), 1));
+        BuilderUtils.newResource(2 * GB, 1), 1));
     fs.allocate(appAttemptId2, ask2, emptyId);
     fs.allocate(appAttemptId2, ask2, emptyId);
     
     
     // Trigger container assignment
     // Trigger container assignment

+ 10 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java

@@ -55,9 +55,9 @@ public class TestResourceManager {
 
 
   private org.apache.hadoop.yarn.server.resourcemanager.NodeManager
   private org.apache.hadoop.yarn.server.resourcemanager.NodeManager
       registerNode(String hostName, int containerManagerPort, int httpPort,
       registerNode(String hostName, int containerManagerPort, int httpPort,
-          String rackName, int memory) throws IOException {
+          String rackName, Resource capability) throws IOException {
     return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
     return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
-        hostName, containerManagerPort, httpPort, rackName, memory,
+        hostName, containerManagerPort, httpPort, rackName, capability,
         resourceManager.getResourceTrackerService(), resourceManager
         resourceManager.getResourceTrackerService(), resourceManager
             .getRMContext());
             .getRMContext());
   }
   }
@@ -71,13 +71,15 @@ public class TestResourceManager {
     // Register node1
     // Register node1
     String host1 = "host1";
     String host1 = "host1";
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm1 = 
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm1 = 
-      registerNode(host1, 1234, 2345, NetworkTopology.DEFAULT_RACK, memory);
+      registerNode(host1, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
+          Resources.createResource(memory, 1));
     nm1.heartbeat();
     nm1.heartbeat();
     
     
     // Register node2
     // Register node2
     String host2 = "host2";
     String host2 = "host2";
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm2 = 
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm2 = 
-      registerNode(host2, 1234, 2345, NetworkTopology.DEFAULT_RACK, memory/2);
+      registerNode(host2, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
+          Resources.createResource(memory/2, 1));
     nm2.heartbeat();
     nm2.heartbeat();
 
 
     // Submit an application
     // Submit an application
@@ -89,7 +91,7 @@ public class TestResourceManager {
     
     
     // Application resource requirements
     // Application resource requirements
     final int memory1 = 1024;
     final int memory1 = 1024;
-    Resource capability1 = Resources.createResource(memory1);
+    Resource capability1 = Resources.createResource(memory1, 1);
     Priority priority1 = 
     Priority priority1 = 
       org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(1);
       org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(1);
     application.addResourceRequestSpec(priority1, capability1);
     application.addResourceRequestSpec(priority1, capability1);
@@ -98,7 +100,7 @@ public class TestResourceManager {
     application.addTask(t1);
     application.addTask(t1);
         
         
     final int memory2 = 2048;
     final int memory2 = 2048;
-    Resource capability2 = Resources.createResource(memory2);
+    Resource capability2 = Resources.createResource(memory2, 1);
     Priority priority0 = 
     Priority priority0 = 
       org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(0); // higher
       org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(0); // higher
     application.addResourceRequestSpec(priority0, capability2);
     application.addResourceRequestSpec(priority0, capability2);
@@ -161,7 +163,8 @@ public class TestResourceManager {
     String host1 = "host1";
     String host1 = "host1";
     final int memory = 4 * 1024;
     final int memory = 4 * 1024;
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm1 = 
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm1 = 
-      registerNode(host1, 1234, 2345, NetworkTopology.DEFAULT_RACK, memory);
+      registerNode(host1, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
+          Resources.createResource(memory, 1));
     nm1.heartbeat();
     nm1.heartbeat();
     nm1.heartbeat();
     nm1.heartbeat();
     Collection<RMNode> values = resourceManager.getRMContext().getRMNodes().values();
     Collection<RMNode> values = resourceManager.getRMContext().getRMNodes().values();

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java

@@ -65,7 +65,7 @@ public class TestRMContainerImpl {
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
     ContainerAllocationExpirer expirer = mock(ContainerAllocationExpirer.class);
     ContainerAllocationExpirer expirer = mock(ContainerAllocationExpirer.class);
 
 
-    Resource resource = BuilderUtils.newResource(512);
+    Resource resource = BuilderUtils.newResource(512, 1);
     Priority priority = BuilderUtils.newPriority(5);
     Priority priority = BuilderUtils.newPriority(5);
 
 
     Container container = BuilderUtils.newContainer(containerId, nodeId,
     Container container = BuilderUtils.newContainer(containerId, nodeId,
@@ -135,7 +135,7 @@ public class TestRMContainerImpl {
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
     ContainerAllocationExpirer expirer = mock(ContainerAllocationExpirer.class);
     ContainerAllocationExpirer expirer = mock(ContainerAllocationExpirer.class);
 
 
-    Resource resource = BuilderUtils.newResource(512);
+    Resource resource = BuilderUtils.newResource(512, 1);
     Priority priority = BuilderUtils.newPriority(5);
     Priority priority = BuilderUtils.newPriority(5);
 
 
     Container container = BuilderUtils.newContainer(containerId, nodeId,
     Container container = BuilderUtils.newContainer(containerId, nodeId,

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

@@ -34,7 +34,6 @@ import org.apache.hadoop.metrics2.MetricsSystem;
 import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
 import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.BuilderUtils;
@@ -67,7 +66,7 @@ public class TestQueueMetrics {
     MetricsSource userSource = userSource(ms, queueName, user);
     MetricsSource userSource = userSource(ms, queueName, user);
     checkApps(queueSource, 1, 1, 0, 0, 0, 0);
     checkApps(queueSource, 1, 1, 0, 0, 0, 0);
 
 
-    metrics.setAvailableResourcesToQueue(Resource.createResource(100*GB));
+    metrics.setAvailableResourcesToQueue(Resources.createResource(100*GB));
     metrics.incrPendingResources(user, 5, Resources.createResource(15*GB));
     metrics.incrPendingResources(user, 5, Resources.createResource(15*GB));
     // Available resources is set externally, as it depends on dynamic
     // Available resources is set externally, as it depends on dynamic
     // configurable cluster/queue resources
     // configurable cluster/queue resources

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

@@ -20,38 +20,77 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 
 
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.DominantResourceCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class TestSchedulerUtils {
 public class TestSchedulerUtils {
 
 
   @Test
   @Test
   public void testNormalizeRequest() {
   public void testNormalizeRequest() {
-    int minMemory = 1024;
+    ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
+    
+    final int minMemory = 1024;
+    Resource minResource = Resources.createResource(minMemory, 0);
+    
     ResourceRequest ask = new ResourceRequestPBImpl();
     ResourceRequest ask = new ResourceRequestPBImpl();
 
 
     // case negative memory
     // case negative memory
-    ask.setCapability(Resource.createResource(-1024));
-    SchedulerUtils.normalizeRequest(ask, minMemory);
+    ask.setCapability(Resources.createResource(-1024));
+    SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
     assertEquals(minMemory, ask.getCapability().getMemory());
     assertEquals(minMemory, ask.getCapability().getMemory());
 
 
     // case zero memory
     // case zero memory
-    ask.setCapability(Resource.createResource(0));
-    SchedulerUtils.normalizeRequest(ask, minMemory);
+    ask.setCapability(Resources.createResource(0));
+    SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
     assertEquals(minMemory, ask.getCapability().getMemory());
     assertEquals(minMemory, ask.getCapability().getMemory());
 
 
     // case memory is a multiple of minMemory
     // case memory is a multiple of minMemory
-    ask.setCapability(Resource.createResource(2 * minMemory));
-    SchedulerUtils.normalizeRequest(ask, minMemory);
+    ask.setCapability(Resources.createResource(2 * minMemory));
+    SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
     assertEquals(2 * minMemory, ask.getCapability().getMemory());
     assertEquals(2 * minMemory, ask.getCapability().getMemory());
 
 
     // case memory is not a multiple of minMemory
     // case memory is not a multiple of minMemory
-    ask.setCapability(Resource.createResource(minMemory + 10));
-    SchedulerUtils.normalizeRequest(ask, minMemory);
+    ask.setCapability(Resources.createResource(minMemory + 10));
+    SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
     assertEquals(2 * minMemory, ask.getCapability().getMemory());
     assertEquals(2 * minMemory, ask.getCapability().getMemory());
 
 
   }
   }
+  
+  @Test
+  public void testNormalizeRequestWithDominantResourceCalculator() {
+    ResourceCalculator resourceCalculator = new DominantResourceCalculator();
+    
+    Resource minResource = Resources.createResource(1024, 1);
+    Resource clusterResource = Resources.createResource(10 * 1024, 10);
+    
+    ResourceRequest ask = new ResourceRequestPBImpl();
+
+    // case negative memory/vcores
+    ask.setCapability(Resources.createResource(-1024, -1));
+    SchedulerUtils.normalizeRequest(
+        ask, resourceCalculator, clusterResource, minResource);
+    assertEquals(minResource, ask.getCapability());
 
 
+    // case zero memory/vcores
+    ask.setCapability(Resources.createResource(0, 0));
+    SchedulerUtils.normalizeRequest(
+        ask, resourceCalculator, clusterResource, minResource);
+    assertEquals(minResource, ask.getCapability());
+    assertEquals(1, ask.getCapability().getVirtualCores());
+    assertEquals(1024, ask.getCapability().getMemory());
+
+    // case non-zero memory & zero cores
+    ask.setCapability(Resources.createResource(1536, 0));
+    SchedulerUtils.normalizeRequest(
+        ask, resourceCalculator, clusterResource, minResource);
+    assertEquals(Resources.createResource(2048, 1), ask.getCapability());
+    assertEquals(1, ask.getCapability().getVirtualCores());
+    assertEquals(2048, ask.getCapability().getMemory());
+  }
 }
 }

+ 60 - 38
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java

@@ -17,8 +17,17 @@
  */
  */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -38,6 +47,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@@ -53,6 +64,8 @@ public class TestApplicationLimits {
 
 
   LeafQueue queue;
   LeafQueue queue;
   
   
+  private final ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
+  
   @Before
   @Before
   public void setUp() throws IOException {
   public void setUp() throws IOException {
     CapacitySchedulerConfiguration csConf = 
     CapacitySchedulerConfiguration csConf = 
@@ -64,23 +77,27 @@ public class TestApplicationLimits {
     CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class);
     CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class);
     when(csContext.getConfiguration()).thenReturn(csConf);
     when(csContext.getConfiguration()).thenReturn(csConf);
     when(csContext.getConf()).thenReturn(conf);
     when(csContext.getConf()).thenReturn(conf);
-    when(csContext.getMinimumResourceCapability()).thenReturn(Resources.createResource(GB));
-    when(csContext.getMaximumResourceCapability()).thenReturn(Resources.createResource(16*GB));
-    when(csContext.getClusterResources()).thenReturn(Resources.createResource(10 * 16 * GB));
-    
+    when(csContext.getMinimumResourceCapability()).
+        thenReturn(Resources.createResource(GB, 1));
+    when(csContext.getMaximumResourceCapability()).
+        thenReturn(Resources.createResource(16*GB, 32));
+    when(csContext.getClusterResources()).
+        thenReturn(Resources.createResource(10 * 16 * GB, 10 * 32));
+    when(csContext.getApplicationComparator()).
+        thenReturn(CapacityScheduler.applicationComparator);
+    when(csContext.getQueueComparator()).
+        thenReturn(CapacityScheduler.queueComparator);
+    when(csContext.getResourceCalculator()).
+        thenReturn(resourceCalculator);
+
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CSQueue root = 
     CSQueue root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
         CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
             queues, queues, 
             queues, queues, 
-            CapacityScheduler.queueComparator, 
-            CapacityScheduler.applicationComparator, 
             TestUtils.spyHook);
             TestUtils.spyHook);
 
 
     
     
-    queue = spy(
-        new LeafQueue(csContext, A, root, 
-                      CapacityScheduler.applicationComparator, null)
-        );
+    queue = spy(new LeafQueue(csContext, A, root, null));
 
 
     // Stub out ACL checks
     // Stub out ACL checks
     doReturn(true).
     doReturn(true).
@@ -132,21 +149,23 @@ public class TestApplicationLimits {
     when(csContext.getConfiguration()).thenReturn(csConf);
     when(csContext.getConfiguration()).thenReturn(csConf);
     when(csContext.getConf()).thenReturn(conf);
     when(csContext.getConf()).thenReturn(conf);
     when(csContext.getMinimumResourceCapability()).
     when(csContext.getMinimumResourceCapability()).
-        thenReturn(Resources.createResource(GB));
+        thenReturn(Resources.createResource(GB, 1));
     when(csContext.getMaximumResourceCapability()).
     when(csContext.getMaximumResourceCapability()).
-        thenReturn(Resources.createResource(16*GB));
+        thenReturn(Resources.createResource(16*GB, 16));
+    when(csContext.getApplicationComparator()).
+        thenReturn(CapacityScheduler.applicationComparator);
+    when(csContext.getQueueComparator()).
+        thenReturn(CapacityScheduler.queueComparator);
+    when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     
     
     // Say cluster has 100 nodes of 16G each
     // Say cluster has 100 nodes of 16G each
-    Resource clusterResource = Resources.createResource(100 * 16 * GB);
+    Resource clusterResource = Resources.createResource(100 * 16 * GB, 100 * 16);
     when(csContext.getClusterResources()).thenReturn(clusterResource);
     when(csContext.getClusterResources()).thenReturn(clusterResource);
     
     
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CSQueue root = 
     CSQueue root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
         CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
-            queues, queues, 
-            CapacityScheduler.queueComparator, 
-            CapacityScheduler.applicationComparator,
-            TestUtils.spyHook);
+            queues, queues, TestUtils.spyHook);
 
 
     LeafQueue queue = (LeafQueue)queues.get(A);
     LeafQueue queue = (LeafQueue)queues.get(A);
     
     
@@ -205,8 +224,12 @@ public class TestApplicationLimits {
         );
         );
 
 
     // should return -1 if per queue setting not set
     // should return -1 if per queue setting not set
-    assertEquals((int)csConf.UNDEFINED, csConf.getMaximumApplicationsPerQueue(queue.getQueuePath()));
-    int expectedMaxApps =  (int)(csConf.DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS * 
+    assertEquals(
+        (int)CapacitySchedulerConfiguration.UNDEFINED, 
+        csConf.getMaximumApplicationsPerQueue(queue.getQueuePath()));
+    int expectedMaxApps =  
+        (int)
+        (CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS * 
         queue.getAbsoluteCapacity());
         queue.getAbsoluteCapacity());
     assertEquals(expectedMaxApps, queue.getMaxApplications());
     assertEquals(expectedMaxApps, queue.getMaxApplications());
 
 
@@ -215,8 +238,11 @@ public class TestApplicationLimits {
     assertEquals(expectedMaxAppsPerUser, queue.getMaxApplicationsPerUser());
     assertEquals(expectedMaxAppsPerUser, queue.getMaxApplicationsPerUser());
 
 
     // should default to global setting if per queue setting not set
     // should default to global setting if per queue setting not set
-    assertEquals((long) csConf.DEFAULT_MAXIMUM_APPLICATIONMASTERS_RESOURCE_PERCENT, 
-        (long) csConf.getMaximumApplicationMasterResourcePerQueuePercent(queue.getQueuePath()));
+    assertEquals(
+        (long)CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_APPLICATIONMASTERS_RESOURCE_PERCENT, 
+        (long)csConf.getMaximumApplicationMasterResourcePerQueuePercent(
+            queue.getQueuePath())
+            );
 
 
     // Change the per-queue max AM resources percentage.
     // Change the per-queue max AM resources percentage.
     csConf.setFloat(
     csConf.setFloat(
@@ -228,10 +254,7 @@ public class TestApplicationLimits {
     queues = new HashMap<String, CSQueue>();
     queues = new HashMap<String, CSQueue>();
     root = 
     root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
         CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
-            queues, queues, 
-            CapacityScheduler.queueComparator, 
-            CapacityScheduler.applicationComparator,
-            TestUtils.spyHook);
+            queues, queues, TestUtils.spyHook);
     clusterResource = Resources.createResource(100 * 16 * GB);
     clusterResource = Resources.createResource(100 * 16 * GB);
 
 
     queue = (LeafQueue)queues.get(A);
     queue = (LeafQueue)queues.get(A);
@@ -257,10 +280,7 @@ public class TestApplicationLimits {
     queues = new HashMap<String, CSQueue>();
     queues = new HashMap<String, CSQueue>();
     root = 
     root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
         CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
-            queues, queues, 
-            CapacityScheduler.queueComparator, 
-            CapacityScheduler.applicationComparator,
-            TestUtils.spyHook);
+            queues, queues, TestUtils.spyHook);
 
 
     queue = (LeafQueue)queues.get(A);
     queue = (LeafQueue)queues.get(A);
     assertEquals(9999, (int)csConf.getMaximumApplicationsPerQueue(queue.getQueuePath()));
     assertEquals(9999, (int)csConf.getMaximumApplicationsPerQueue(queue.getQueuePath()));
@@ -445,6 +465,11 @@ public class TestApplicationLimits {
         thenReturn(Resources.createResource(GB));
         thenReturn(Resources.createResource(GB));
     when(csContext.getMaximumResourceCapability()).
     when(csContext.getMaximumResourceCapability()).
         thenReturn(Resources.createResource(16*GB));
         thenReturn(Resources.createResource(16*GB));
+    when(csContext.getApplicationComparator()).
+        thenReturn(CapacityScheduler.applicationComparator);
+    when(csContext.getQueueComparator()).
+        thenReturn(CapacityScheduler.queueComparator);
+    when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
     
     
     // Say cluster has 100 nodes of 16G each
     // Say cluster has 100 nodes of 16G each
     Resource clusterResource = Resources.createResource(100 * 16 * GB);
     Resource clusterResource = Resources.createResource(100 * 16 * GB);
@@ -452,10 +477,7 @@ public class TestApplicationLimits {
     
     
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
     CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
     CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
-        queues, queues, 
-        CapacityScheduler.queueComparator, 
-        CapacityScheduler.applicationComparator, 
-        TestUtils.spyHook);
+        queues, queues, TestUtils.spyHook);
 
 
     // Manipulate queue 'a'
     // Manipulate queue 'a'
     LeafQueue queue = TestLeafQueue.stubLeafQueue((LeafQueue)queues.get(A));
     LeafQueue queue = TestLeafQueue.stubLeafQueue((LeafQueue)queues.get(A));
@@ -490,7 +512,7 @@ public class TestApplicationLimits {
 
 
     // Schedule to compute 
     // Schedule to compute 
     queue.assignContainers(clusterResource, node_0);
     queue.assignContainers(clusterResource, node_0);
-    Resource expectedHeadroom = Resources.createResource(10*16*GB);
+    Resource expectedHeadroom = Resources.createResource(10*16*GB, 1);
     verify(app_0_0).setHeadroom(eq(expectedHeadroom));
     verify(app_0_0).setHeadroom(eq(expectedHeadroom));
 
 
     // Submit second application from user_0, check headroom
     // Submit second application from user_0, check headroom
@@ -528,7 +550,7 @@ public class TestApplicationLimits {
     
     
     // Schedule to compute 
     // Schedule to compute 
     queue.assignContainers(clusterResource, node_0); // Schedule to compute
     queue.assignContainers(clusterResource, node_0); // Schedule to compute
-    expectedHeadroom = Resources.createResource(10*16*GB / 2); // changes
+    expectedHeadroom = Resources.createResource(10*16*GB / 2, 1); // changes
     verify(app_0_0).setHeadroom(eq(expectedHeadroom));
     verify(app_0_0).setHeadroom(eq(expectedHeadroom));
     verify(app_0_1).setHeadroom(eq(expectedHeadroom));
     verify(app_0_1).setHeadroom(eq(expectedHeadroom));
     verify(app_1_0).setHeadroom(eq(expectedHeadroom));
     verify(app_1_0).setHeadroom(eq(expectedHeadroom));
@@ -536,7 +558,7 @@ public class TestApplicationLimits {
     // Now reduce cluster size and check for the smaller headroom
     // Now reduce cluster size and check for the smaller headroom
     clusterResource = Resources.createResource(90*16*GB);
     clusterResource = Resources.createResource(90*16*GB);
     queue.assignContainers(clusterResource, node_0); // Schedule to compute
     queue.assignContainers(clusterResource, node_0); // Schedule to compute
-    expectedHeadroom = Resources.createResource(9*16*GB / 2); // changes
+    expectedHeadroom = Resources.createResource(9*16*GB / 2, 1); // changes
     verify(app_0_0).setHeadroom(eq(expectedHeadroom));
     verify(app_0_0).setHeadroom(eq(expectedHeadroom));
     verify(app_0_1).setHeadroom(eq(expectedHeadroom));
     verify(app_0_1).setHeadroom(eq(expectedHeadroom));
     verify(app_1_0).setHeadroom(eq(expectedHeadroom));
     verify(app_1_0).setHeadroom(eq(expectedHeadroom));

+ 10 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java

@@ -91,10 +91,10 @@ public class TestCapacityScheduler {
   
   
   private org.apache.hadoop.yarn.server.resourcemanager.NodeManager
   private org.apache.hadoop.yarn.server.resourcemanager.NodeManager
       registerNode(String hostName, int containerManagerPort, int httpPort,
       registerNode(String hostName, int containerManagerPort, int httpPort,
-          String rackName, int memory)
+          String rackName, Resource capability)
           throws IOException {
           throws IOException {
     return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
     return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
-        hostName, containerManagerPort, httpPort, rackName, memory,
+        hostName, containerManagerPort, httpPort, rackName, capability,
         resourceManager.getResourceTrackerService(), resourceManager
         resourceManager.getResourceTrackerService(), resourceManager
             .getRMContext());
             .getRMContext());
   }  
   }  
@@ -107,13 +107,15 @@ public class TestCapacityScheduler {
     // Register node1
     // Register node1
     String host_0 = "host_0";
     String host_0 = "host_0";
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_0 = 
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_0 = 
-      registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK, 4 * GB);
+      registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
+          Resources.createResource(4 * GB, 1));
     nm_0.heartbeat();
     nm_0.heartbeat();
     
     
     // Register node2
     // Register node2
     String host_1 = "host_1";
     String host_1 = "host_1";
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_1 = 
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_1 = 
-      registerNode(host_1, 1234, 2345, NetworkTopology.DEFAULT_RACK, 2 * GB);
+      registerNode(host_1, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
+          Resources.createResource(2 * GB, 1));
     nm_1.heartbeat();
     nm_1.heartbeat();
 
 
     // ResourceRequest priorities
     // ResourceRequest priorities
@@ -129,10 +131,10 @@ public class TestCapacityScheduler {
     application_0.addNodeManager(host_0, 1234, nm_0);
     application_0.addNodeManager(host_0, 1234, nm_0);
     application_0.addNodeManager(host_1, 1234, nm_1);
     application_0.addNodeManager(host_1, 1234, nm_1);
 
 
-    Resource capability_0_0 = Resources.createResource(1 * GB);
+    Resource capability_0_0 = Resources.createResource(1 * GB, 1);
     application_0.addResourceRequestSpec(priority_1, capability_0_0);
     application_0.addResourceRequestSpec(priority_1, capability_0_0);
     
     
-    Resource capability_0_1 = Resources.createResource(2 * GB);
+    Resource capability_0_1 = Resources.createResource(2 * GB, 1);
     application_0.addResourceRequestSpec(priority_0, capability_0_1);
     application_0.addResourceRequestSpec(priority_0, capability_0_1);
 
 
     Task task_0_0 = new Task(application_0, priority_1, 
     Task task_0_0 = new Task(application_0, priority_1, 
@@ -146,10 +148,10 @@ public class TestCapacityScheduler {
     application_1.addNodeManager(host_0, 1234, nm_0);
     application_1.addNodeManager(host_0, 1234, nm_0);
     application_1.addNodeManager(host_1, 1234, nm_1);
     application_1.addNodeManager(host_1, 1234, nm_1);
     
     
-    Resource capability_1_0 = Resources.createResource(3 * GB);
+    Resource capability_1_0 = Resources.createResource(3 * GB, 1);
     application_1.addResourceRequestSpec(priority_1, capability_1_0);
     application_1.addResourceRequestSpec(priority_1, capability_1_0);
     
     
-    Resource capability_1_1 = Resources.createResource(2 * GB);
+    Resource capability_1_1 = Resources.createResource(2 * GB, 1);
     application_1.addResourceRequestSpec(priority_0, capability_1_1);
     application_1.addResourceRequestSpec(priority_0, capability_1_1);
 
 
     Task task_1_0 = new Task(application_1, priority_1, 
     Task task_1_0 = new Task(application_1, priority_1, 

+ 38 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java

@@ -55,6 +55,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -90,6 +92,8 @@ public class TestLeafQueue {
   final static int GB = 1024;
   final static int GB = 1024;
   final static String DEFAULT_RACK = "/default";
   final static String DEFAULT_RACK = "/default";
 
 
+  private final ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
+  
   @Before
   @Before
   public void setUp() throws Exception {
   public void setUp() throws Exception {
     CapacityScheduler spyCs = new CapacityScheduler();
     CapacityScheduler spyCs = new CapacityScheduler();
@@ -108,17 +112,22 @@ public class TestLeafQueue {
     when(csContext.getConfiguration()).thenReturn(csConf);
     when(csContext.getConfiguration()).thenReturn(csConf);
     when(csContext.getConf()).thenReturn(conf);
     when(csContext.getConf()).thenReturn(conf);
     when(csContext.getMinimumResourceCapability()).
     when(csContext.getMinimumResourceCapability()).
-        thenReturn(Resources.createResource(GB));
+        thenReturn(Resources.createResource(GB, 1));
     when(csContext.getMaximumResourceCapability()).
     when(csContext.getMaximumResourceCapability()).
-        thenReturn(Resources.createResource(16*GB));
+        thenReturn(Resources.createResource(16*GB, 32));
     when(csContext.getClusterResources()).
     when(csContext.getClusterResources()).
-        thenReturn(Resources.createResource(100 * 16 * GB));
+        thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
+    when(csContext.getApplicationComparator()).
+    thenReturn(CapacityScheduler.applicationComparator);
+    when(csContext.getQueueComparator()).
+        thenReturn(CapacityScheduler.queueComparator);
+    when(csContext.getResourceCalculator()).
+        thenReturn(resourceCalculator);
+
     root = 
     root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
             CapacitySchedulerConfiguration.ROOT, 
             CapacitySchedulerConfiguration.ROOT, 
             queues, queues, 
             queues, queues, 
-            CapacityScheduler.queueComparator, 
-            CapacityScheduler.applicationComparator, 
             TestUtils.spyHook);
             TestUtils.spyHook);
 
 
     cs.reinitialize(csConf, rmContext);
     cs.reinitialize(csConf, rmContext);
@@ -266,7 +275,8 @@ public class TestLeafQueue {
     FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB);
     FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB);
     
     
     final int numNodes = 1;
     final int numNodes = 1;
-    Resource clusterResource = Resources.createResource(numNodes * (8*GB));
+    Resource clusterResource = 
+        Resources.createResource(numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
 
 
     // Setup resource-requests
     // Setup resource-requests
@@ -387,7 +397,8 @@ public class TestLeafQueue {
     FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB);
     FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB);
     
     
     final int numNodes = 1;
     final int numNodes = 1;
-    Resource clusterResource = Resources.createResource(numNodes * (8*GB));
+    Resource clusterResource = 
+        Resources.createResource(numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
 
 
     // Setup resource-requests
     // Setup resource-requests
@@ -519,7 +530,8 @@ public class TestLeafQueue {
     FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 8*GB);
     FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 8*GB);
     
     
     final int numNodes = 2;
     final int numNodes = 2;
-    Resource clusterResource = Resources.createResource(numNodes * (8*GB));
+    Resource clusterResource = 
+        Resources.createResource(numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
  
  
     // Setup resource-requests
     // Setup resource-requests
@@ -612,7 +624,7 @@ public class TestLeafQueue {
     FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 8*GB);
     FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 8*GB);
     
     
     final int numNodes = 2;
     final int numNodes = 2;
-    Resource clusterResource = Resources.createResource(numNodes * (8*GB));
+    Resource clusterResource = Resources.createResource(numNodes * (8*GB), 1);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
  
  
     // Setup resource-requests
     // Setup resource-requests
@@ -728,7 +740,8 @@ public class TestLeafQueue {
     FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB);
     FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB);
     
     
     final int numNodes = 1;
     final int numNodes = 1;
-    Resource clusterResource = Resources.createResource(numNodes * (8*GB));
+    Resource clusterResource = 
+        Resources.createResource(numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     
     
     // Setup resource-requests
     // Setup resource-requests
@@ -889,7 +902,8 @@ public class TestLeafQueue {
     FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 4*GB);
     FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 4*GB);
     
     
     final int numNodes = 2;
     final int numNodes = 2;
-    Resource clusterResource = Resources.createResource(numNodes * (4*GB));
+    Resource clusterResource = 
+        Resources.createResource(numNodes * (4*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     
     
     // Setup resource-requests
     // Setup resource-requests
@@ -990,7 +1004,8 @@ public class TestLeafQueue {
     FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 4*GB);
     FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 4*GB);
 
 
     final int numNodes = 3;
     final int numNodes = 3;
-    Resource clusterResource = Resources.createResource(numNodes * (4*GB));
+    Resource clusterResource = 
+        Resources.createResource(numNodes * (4*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
 
 
     // Setup resource-requests
     // Setup resource-requests
@@ -1090,11 +1105,13 @@ public class TestLeafQueue {
     FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 4*GB);
     FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 4*GB);
     
     
     final int numNodes = 3;
     final int numNodes = 3;
-    Resource clusterResource = Resources.createResource(numNodes * (4*GB));
+    Resource clusterResource = 
+        Resources.createResource(numNodes * (4*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getMaximumResourceCapability()).thenReturn(
     when(csContext.getMaximumResourceCapability()).thenReturn(
-        Resources.createResource(4*GB));
-    when(a.getMaximumAllocation()).thenReturn(Resources.createResource(4*GB));
+        Resources.createResource(4*GB, 16));
+    when(a.getMaximumAllocation()).thenReturn(
+        Resources.createResource(4*GB, 16));
     when(a.getMinimumAllocationFactor()).thenReturn(0.25f); // 1G / 4G 
     when(a.getMinimumAllocationFactor()).thenReturn(0.25f); // 1G / 4G 
     
     
     // Setup resource-requests
     // Setup resource-requests
@@ -1204,7 +1221,8 @@ public class TestLeafQueue {
     FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, rack_2, 0, 8*GB);
     FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, rack_2, 0, 8*GB);
 
 
     final int numNodes = 3;
     final int numNodes = 3;
-    Resource clusterResource = Resources.createResource(numNodes * (8*GB));
+    Resource clusterResource = 
+        Resources.createResource(numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     
     
     // Setup resource-requests and submit
     // Setup resource-requests and submit
@@ -1344,7 +1362,8 @@ public class TestLeafQueue {
     FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, rack_2, 0, 8*GB);
     FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, rack_2, 0, 8*GB);
 
 
     final int numNodes = 3;
     final int numNodes = 3;
-    Resource clusterResource = Resources.createResource(numNodes * (8*GB));
+    Resource clusterResource = 
+        Resources.createResource(numNodes * (8*GB), 1);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     
     
     // Setup resource-requests and submit
     // Setup resource-requests and submit
@@ -1473,7 +1492,8 @@ public class TestLeafQueue {
     FiCaSchedulerNode node_1_0 = TestUtils.getMockNode(host_1_0, rack_1, 0, 8*GB);
     FiCaSchedulerNode node_1_0 = TestUtils.getMockNode(host_1_0, rack_1, 0, 8*GB);
     
     
     final int numNodes = 3;
     final int numNodes = 3;
-    Resource clusterResource = Resources.createResource(numNodes * (8*GB));
+    Resource clusterResource = Resources.createResource(
+        numNodes * (8*GB), numNodes * 16);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
 
 
     // Setup resource-requests and submit
     // Setup resource-requests and submit

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

@@ -45,6 +45,8 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@@ -68,6 +70,9 @@ public class TestParentQueue {
   final static int GB = 1024;
   final static int GB = 1024;
   final static String DEFAULT_RACK = "/default";
   final static String DEFAULT_RACK = "/default";
 
 
+  private final ResourceCalculator resourceComparator =
+      new DefaultResourceCalculator();
+  
   @Before
   @Before
   public void setUp() throws Exception {
   public void setUp() throws Exception {
     rmContext = TestUtils.getMockRMContext();
     rmContext = TestUtils.getMockRMContext();
@@ -78,11 +83,17 @@ public class TestParentQueue {
     when(csContext.getConf()).thenReturn(conf);
     when(csContext.getConf()).thenReturn(conf);
     when(csContext.getConfiguration()).thenReturn(csConf);
     when(csContext.getConfiguration()).thenReturn(csConf);
     when(csContext.getMinimumResourceCapability()).thenReturn(
     when(csContext.getMinimumResourceCapability()).thenReturn(
-        Resources.createResource(GB));
+        Resources.createResource(GB, 1));
     when(csContext.getMaximumResourceCapability()).thenReturn(
     when(csContext.getMaximumResourceCapability()).thenReturn(
-        Resources.createResource(16*GB));
+        Resources.createResource(16*GB, 32));
     when(csContext.getClusterResources()).
     when(csContext.getClusterResources()).
-        thenReturn(Resources.createResource(100 * 16 * GB));
+        thenReturn(Resources.createResource(100 * 16 * GB, 100 * 32));
+    when(csContext.getApplicationComparator()).
+    thenReturn(CapacityScheduler.applicationComparator);
+    when(csContext.getQueueComparator()).
+    thenReturn(CapacityScheduler.queueComparator);
+    when(csContext.getResourceCalculator()).
+    thenReturn(resourceComparator);
   }
   }
   
   
   private static final String A = "a";
   private static final String A = "a";
@@ -104,7 +115,7 @@ public class TestParentQueue {
   private FiCaSchedulerApp getMockApplication(int appId, String user) {
   private FiCaSchedulerApp getMockApplication(int appId, String user) {
     FiCaSchedulerApp application = mock(FiCaSchedulerApp.class);
     FiCaSchedulerApp application = mock(FiCaSchedulerApp.class);
     doReturn(user).when(application).getUser();
     doReturn(user).when(application).getUser();
-    doReturn(Resources.createResource(0)).when(application).getHeadroom();
+    doReturn(Resources.createResource(0, 0)).when(application).getHeadroom();
     return application;
     return application;
   }
   }
 
 
@@ -192,12 +203,11 @@ public class TestParentQueue {
     CSQueue root = 
     CSQueue root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
             CapacitySchedulerConfiguration.ROOT, queues, queues, 
             CapacitySchedulerConfiguration.ROOT, queues, queues, 
-            CapacityScheduler.queueComparator, 
-            CapacityScheduler.applicationComparator,
             TestUtils.spyHook);
             TestUtils.spyHook);
 
 
     // Setup some nodes
     // Setup some nodes
     final int memoryPerNode = 10;
     final int memoryPerNode = 10;
+    final int coresPerNode = 16;
     final int numNodes = 2;
     final int numNodes = 2;
     
     
     FiCaSchedulerNode node_0 = 
     FiCaSchedulerNode node_0 = 
@@ -206,7 +216,8 @@ public class TestParentQueue {
         TestUtils.getMockNode("host_1", DEFAULT_RACK, 0, memoryPerNode*GB);
         TestUtils.getMockNode("host_1", DEFAULT_RACK, 0, memoryPerNode*GB);
     
     
     final Resource clusterResource = 
     final Resource clusterResource = 
-        Resources.createResource(numNodes * (memoryPerNode*GB));
+        Resources.createResource(numNodes * (memoryPerNode*GB),
+            numNodes * coresPerNode);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
 
 
     // Start testing
     // Start testing
@@ -286,8 +297,7 @@ public class TestParentQueue {
     try {
     try {
       CapacityScheduler.parseQueue(csContext, csConf, null,
       CapacityScheduler.parseQueue(csContext, csConf, null,
           CapacitySchedulerConfiguration.ROOT, queues, queues,
           CapacitySchedulerConfiguration.ROOT, queues, queues,
-          CapacityScheduler.queueComparator,
-          CapacityScheduler.applicationComparator, TestUtils.spyHook);
+          TestUtils.spyHook);
     } catch (IllegalArgumentException ie) {
     } catch (IllegalArgumentException ie) {
       exceptionOccured = true;
       exceptionOccured = true;
     }
     }
@@ -301,8 +311,7 @@ public class TestParentQueue {
     try {
     try {
       CapacityScheduler.parseQueue(csContext, csConf, null,
       CapacityScheduler.parseQueue(csContext, csConf, null,
           CapacitySchedulerConfiguration.ROOT, queues, queues,
           CapacitySchedulerConfiguration.ROOT, queues, queues,
-          CapacityScheduler.queueComparator,
-          CapacityScheduler.applicationComparator, TestUtils.spyHook);
+          TestUtils.spyHook);
     } catch (IllegalArgumentException ie) {
     } catch (IllegalArgumentException ie) {
       exceptionOccured = true;
       exceptionOccured = true;
     }
     }
@@ -316,8 +325,7 @@ public class TestParentQueue {
     try {
     try {
       CapacityScheduler.parseQueue(csContext, csConf, null,
       CapacityScheduler.parseQueue(csContext, csConf, null,
           CapacitySchedulerConfiguration.ROOT, queues, queues,
           CapacitySchedulerConfiguration.ROOT, queues, queues,
-          CapacityScheduler.queueComparator,
-          CapacityScheduler.applicationComparator, TestUtils.spyHook);
+          TestUtils.spyHook);
     } catch (IllegalArgumentException ie) {
     } catch (IllegalArgumentException ie) {
       exceptionOccured = true;
       exceptionOccured = true;
     }
     }
@@ -394,12 +402,11 @@ public class TestParentQueue {
     CSQueue root = 
     CSQueue root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
             CapacitySchedulerConfiguration.ROOT, queues, queues, 
             CapacitySchedulerConfiguration.ROOT, queues, queues, 
-            CapacityScheduler.queueComparator, 
-            CapacityScheduler.applicationComparator,
             TestUtils.spyHook);
             TestUtils.spyHook);
     
     
     // Setup some nodes
     // Setup some nodes
     final int memoryPerNode = 10;
     final int memoryPerNode = 10;
+    final int coresPerNode = 16;
     final int numNodes = 3;
     final int numNodes = 3;
     
     
     FiCaSchedulerNode node_0 = 
     FiCaSchedulerNode node_0 = 
@@ -410,7 +417,8 @@ public class TestParentQueue {
         TestUtils.getMockNode("host_2", DEFAULT_RACK, 0, memoryPerNode*GB);
         TestUtils.getMockNode("host_2", DEFAULT_RACK, 0, memoryPerNode*GB);
     
     
     final Resource clusterResource = 
     final Resource clusterResource = 
-        Resources.createResource(numNodes * (memoryPerNode*GB));
+        Resources.createResource(numNodes * (memoryPerNode*GB), 
+            numNodes * coresPerNode);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
 
 
     // Start testing
     // Start testing
@@ -510,8 +518,6 @@ public class TestParentQueue {
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>(); 
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>(); 
     CapacityScheduler.parseQueue(csContext, csConf, null, 
     CapacityScheduler.parseQueue(csContext, csConf, null, 
         CapacitySchedulerConfiguration.ROOT, queues, queues, 
         CapacitySchedulerConfiguration.ROOT, queues, queues, 
-        CapacityScheduler.queueComparator, 
-        CapacityScheduler.applicationComparator,
         TestUtils.spyHook);
         TestUtils.spyHook);
   }
   }
   
   
@@ -529,8 +535,6 @@ public class TestParentQueue {
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>(); 
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>(); 
     CapacityScheduler.parseQueue(csContext, csConf, null, 
     CapacityScheduler.parseQueue(csContext, csConf, null, 
         CapacitySchedulerConfiguration.ROOT, queues, queues, 
         CapacitySchedulerConfiguration.ROOT, queues, queues, 
-        CapacityScheduler.queueComparator, 
-        CapacityScheduler.applicationComparator,
         TestUtils.spyHook);
         TestUtils.spyHook);
   }
   }
   
   
@@ -553,8 +557,6 @@ public class TestParentQueue {
     try {
     try {
       CapacityScheduler.parseQueue(csContext, csConf, null, 
       CapacityScheduler.parseQueue(csContext, csConf, null, 
           CapacitySchedulerConfiguration.ROOT, queues, queues, 
           CapacitySchedulerConfiguration.ROOT, queues, queues, 
-          CapacityScheduler.queueComparator, 
-          CapacityScheduler.applicationComparator,
           TestUtils.spyHook);
           TestUtils.spyHook);
     } catch (IllegalArgumentException e) {
     } catch (IllegalArgumentException e) {
       fail("Failed to create queues with 0 capacity: " + e);
       fail("Failed to create queues with 0 capacity: " + e);
@@ -571,12 +573,11 @@ public class TestParentQueue {
     CSQueue root = 
     CSQueue root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
             CapacitySchedulerConfiguration.ROOT, queues, queues, 
             CapacitySchedulerConfiguration.ROOT, queues, queues, 
-            CapacityScheduler.queueComparator, 
-            CapacityScheduler.applicationComparator,
             TestUtils.spyHook);
             TestUtils.spyHook);
 
 
     // Setup some nodes
     // Setup some nodes
     final int memoryPerNode = 10;
     final int memoryPerNode = 10;
+    final int coresPerNode = 16;
     final int numNodes = 2;
     final int numNodes = 2;
     
     
     FiCaSchedulerNode node_0 = 
     FiCaSchedulerNode node_0 = 
@@ -585,7 +586,8 @@ public class TestParentQueue {
         TestUtils.getMockNode("host_1", DEFAULT_RACK, 0, memoryPerNode*GB);
         TestUtils.getMockNode("host_1", DEFAULT_RACK, 0, memoryPerNode*GB);
     
     
     final Resource clusterResource = 
     final Resource clusterResource = 
-        Resources.createResource(numNodes * (memoryPerNode*GB));
+        Resources.createResource(numNodes * (memoryPerNode*GB), 
+            numNodes * coresPerNode);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
 
 
     // Start testing
     // Start testing
@@ -637,12 +639,11 @@ public class TestParentQueue {
     CSQueue root = 
     CSQueue root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
             CapacitySchedulerConfiguration.ROOT, queues, queues, 
             CapacitySchedulerConfiguration.ROOT, queues, queues, 
-            CapacityScheduler.queueComparator, 
-            CapacityScheduler.applicationComparator,
             TestUtils.spyHook);
             TestUtils.spyHook);
 
 
     // Setup some nodes
     // Setup some nodes
     final int memoryPerNode = 10;
     final int memoryPerNode = 10;
+    final int coresPerNode = 10;
     final int numNodes = 2;
     final int numNodes = 2;
     
     
     FiCaSchedulerNode node_0 = 
     FiCaSchedulerNode node_0 = 
@@ -651,7 +652,8 @@ public class TestParentQueue {
         TestUtils.getMockNode("host_1", DEFAULT_RACK, 0, memoryPerNode*GB);
         TestUtils.getMockNode("host_1", DEFAULT_RACK, 0, memoryPerNode*GB);
     
     
     final Resource clusterResource = 
     final Resource clusterResource = 
-        Resources.createResource(numNodes * (memoryPerNode*GB));
+        Resources.createResource(numNodes * (memoryPerNode*GB),
+            numNodes * coresPerNode);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
 
 
     // Start testing
     // Start testing
@@ -721,8 +723,6 @@ public class TestParentQueue {
     CSQueue root = 
     CSQueue root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
             CapacitySchedulerConfiguration.ROOT, queues, queues, 
             CapacitySchedulerConfiguration.ROOT, queues, queues, 
-            CapacityScheduler.queueComparator, 
-            CapacityScheduler.applicationComparator,
             TestUtils.spyHook);
             TestUtils.spyHook);
 
 
     UserGroupInformation user = UserGroupInformation.getCurrentUser();
     UserGroupInformation user = UserGroupInformation.getCurrentUser();

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

@@ -118,7 +118,7 @@ public class TestUtils {
       RecordFactory recordFactory) {
       RecordFactory recordFactory) {
     ResourceRequest request = 
     ResourceRequest request = 
         recordFactory.newRecordInstance(ResourceRequest.class);
         recordFactory.newRecordInstance(ResourceRequest.class);
-    Resource capability = Resources.createResource(memory);
+    Resource capability = Resources.createResource(memory, 1);
     
     
     request.setNumContainers(numContainers);
     request.setNumContainers(numContainers);
     request.setHostName(hostName);
     request.setHostName(hostName);
@@ -153,7 +153,7 @@ public class TestUtils {
     RMNode rmNode = mock(RMNode.class);
     RMNode rmNode = mock(RMNode.class);
     when(rmNode.getNodeID()).thenReturn(nodeId);
     when(rmNode.getNodeID()).thenReturn(nodeId);
     when(rmNode.getTotalCapability()).thenReturn(
     when(rmNode.getTotalCapability()).thenReturn(
-        Resources.createResource(capability));
+        Resources.createResource(capability, 1));
     when(rmNode.getNodeAddress()).thenReturn(host+":"+port);
     when(rmNode.getNodeAddress()).thenReturn(host+":"+port);
     when(rmNode.getHostName()).thenReturn(host);
     when(rmNode.getHostName()).thenReturn(host);
     when(rmNode.getRackName()).thenReturn(rack);
     when(rmNode.getRackName()).thenReturn(rack);

+ 6 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java

@@ -71,9 +71,9 @@ public class TestFifoScheduler {
   
   
   private org.apache.hadoop.yarn.server.resourcemanager.NodeManager
   private org.apache.hadoop.yarn.server.resourcemanager.NodeManager
       registerNode(String hostName, int containerManagerPort, int nmHttpPort,
       registerNode(String hostName, int containerManagerPort, int nmHttpPort,
-          String rackName, int memory) throws IOException {
+          String rackName, Resource capability) throws IOException {
     return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
     return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
-        hostName, containerManagerPort, nmHttpPort, rackName, memory,
+        hostName, containerManagerPort, nmHttpPort, rackName, capability,
         resourceManager.getResourceTrackerService(), resourceManager
         resourceManager.getResourceTrackerService(), resourceManager
             .getRMContext());
             .getRMContext());
   }
   }
@@ -121,13 +121,15 @@ public class TestFifoScheduler {
     // Register node1
     // Register node1
     String host_0 = "host_0";
     String host_0 = "host_0";
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_0 = 
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_0 = 
-      registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK, 4 * GB);
+      registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
+          Resources.createResource(4 * GB, 1));
     nm_0.heartbeat();
     nm_0.heartbeat();
     
     
     // Register node2
     // Register node2
     String host_1 = "host_1";
     String host_1 = "host_1";
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_1 = 
     org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_1 = 
-      registerNode(host_1, 1234, 2345, NetworkTopology.DEFAULT_RACK, 2 * GB);
+      registerNode(host_1, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
+          Resources.createResource(2 * GB, 1));
     nm_1.heartbeat();
     nm_1.heartbeat();
 
 
     // ResourceRequest priorities
     // ResourceRequest priorities

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java

@@ -424,8 +424,8 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
     assertEquals("absoluteUsedCapacity doesn't match",
     assertEquals("absoluteUsedCapacity doesn't match",
         0, info.absoluteUsedCapacity, 1e-3f);
         0, info.absoluteUsedCapacity, 1e-3f);
     assertEquals("numApplications doesn't match", 0, info.numApplications);
     assertEquals("numApplications doesn't match", 0, info.numApplications);
-    assertTrue("usedResources doesn't match",
-        info.usedResources.matches("memory: 0"));
+    assertTrue("usedResources doesn't match ",
+        info.usedResources.matches("<memory:0, vCores:0>"));
     assertTrue("queueName doesn't match, got: " + info.queueName
     assertTrue("queueName doesn't match, got: " + info.queueName
         + " expected: " + q, qshortName.matches(info.queueName));
         + " expected: " + q, qshortName.matches(info.queueName));
     assertTrue("state doesn't match",
     assertTrue("state doesn't match",

+ 6 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java

@@ -218,7 +218,7 @@ public class TestContainerManagerSecurity {
     dummyIdentifier.readFields(di);
     dummyIdentifier.readFields(di);
 
 
     // Malice user modifies the resource amount
     // Malice user modifies the resource amount
-    Resource modifiedResource = BuilderUtils.newResource(2048);
+    Resource modifiedResource = BuilderUtils.newResource(2048, 1);
     ContainerTokenIdentifier modifiedIdentifier =
     ContainerTokenIdentifier modifiedIdentifier =
         new ContainerTokenIdentifier(dummyIdentifier.getContainerID(),
         new ContainerTokenIdentifier(dummyIdentifier.getContainerID(),
           dummyIdentifier.getNmHostAddress(), "testUser", modifiedResource,
           dummyIdentifier.getNmHostAddress(), "testUser", modifiedResource,
@@ -403,7 +403,7 @@ public class TestContainerManagerSecurity {
 
 
     ContainerLaunchContext amContainer = BuilderUtils
     ContainerLaunchContext amContainer = BuilderUtils
         .newContainerLaunchContext(null, "testUser", BuilderUtils
         .newContainerLaunchContext(null, "testUser", BuilderUtils
-            .newResource(1024), Collections.<String, LocalResource>emptyMap(),
+            .newResource(1024, 1), Collections.<String, LocalResource>emptyMap(),
             new HashMap<String, String>(), Arrays.asList("sleep", "100"),
             new HashMap<String, String>(), Arrays.asList("sleep", "100"),
             new HashMap<String, ByteBuffer>(), null,
             new HashMap<String, ByteBuffer>(), null,
             new HashMap<ApplicationAccessType, String>());
             new HashMap<ApplicationAccessType, String>());
@@ -481,7 +481,7 @@ public class TestContainerManagerSecurity {
     // Request a container allocation.
     // Request a container allocation.
     List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
     List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
     ask.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0), "*",
     ask.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0), "*",
-        BuilderUtils.newResource(1024), 1));
+        BuilderUtils.newResource(1024, 1), 1));
 
 
     AllocateRequest allocateRequest = BuilderUtils.newAllocateRequest(
     AllocateRequest allocateRequest = BuilderUtils.newAllocateRequest(
         BuilderUtils.newApplicationAttemptId(appID, 1), 0, 0F, ask,
         BuilderUtils.newApplicationAttemptId(appID, 1), 0, 0F, ask,
@@ -582,7 +582,9 @@ public class TestContainerManagerSecurity {
     ContainerLaunchContext context =
     ContainerLaunchContext context =
         BuilderUtils.newContainerLaunchContext(tokenId.getContainerID(),
         BuilderUtils.newContainerLaunchContext(tokenId.getContainerID(),
             "testUser",
             "testUser",
-            BuilderUtils.newResource(tokenId.getResource().getMemory()),
+            BuilderUtils.newResource(
+                tokenId.getResource().getMemory(), 
+                tokenId.getResource().getVirtualCores()),
             new HashMap<String, LocalResource>(),
             new HashMap<String, LocalResource>(),
             new HashMap<String, String>(), new ArrayList<String>(),
             new HashMap<String, String>(), new ArrayList<String>(),
             new HashMap<String, ByteBuffer>(), null,
             new HashMap<String, ByteBuffer>(), null,