You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by zj...@apache.org on 2015/07/13 20:59:47 UTC

[35/48] hadoop git commit: YARN-3800. Reduce storage footprint for ReservationAllocation. Contributed by Anubhav Dhoot.

YARN-3800. Reduce storage footprint for ReservationAllocation. Contributed by Anubhav Dhoot.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fb8b2c18
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fb8b2c18
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fb8b2c18

Branch: refs/heads/YARN-2928
Commit: fb8b2c18b39db251bc847e9da03174a597f7bf33
Parents: 6293fab
Author: carlo curino <Carlo Curino>
Authored: Thu Jul 9 16:47:35 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon Jul 13 11:51:15 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../reservation/GreedyReservationAgent.java     | 27 ++++++-----
 .../reservation/InMemoryPlan.java               |  9 ++--
 .../InMemoryReservationAllocation.java          | 24 +++++----
 .../RLESparseResourceAllocation.java            | 43 ++---------------
 .../reservation/ReservationAllocation.java      |  3 +-
 .../reservation/ReservationSystemUtil.java      | 51 ++++++++++++++++++++
 .../reservation/ReservationSystemTestUtil.java  | 11 +++--
 .../reservation/TestCapacityOverTimePolicy.java | 16 +++---
 .../reservation/TestGreedyReservationAgent.java |  2 +-
 .../reservation/TestInMemoryPlan.java           | 37 ++++++++++----
 .../TestInMemoryReservationAllocation.java      | 29 ++++++-----
 .../TestRLESparseResourceAllocation.java        | 33 ++++++-------
 .../TestSimpleCapacityReplanner.java            | 11 +++--
 14 files changed, 176 insertions(+), 123 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f1114c5..3a78ed4 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -435,6 +435,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3827. Migrate YARN native build to new CMake framework (Alan Burlison
     via Colin P. McCabe)
 
+    YARN-3800. Reduce storage footprint for ReservationAllocation. (Anubhav Dhoot
+    via curino)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/GreedyReservationAgent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/GreedyReservationAgent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/GreedyReservationAgent.java
index 5a61b94..214df1c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/GreedyReservationAgent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/GreedyReservationAgent.java
@@ -97,8 +97,8 @@ public class GreedyReservationAgent implements ReservationAgent {
     long curDeadline = deadline;
     long oldDeadline = -1;
 
-    Map<ReservationInterval, ReservationRequest> allocations =
-        new HashMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> allocations =
+        new HashMap<ReservationInterval, Resource>();
     RLESparseResourceAllocation tempAssigned =
         new RLESparseResourceAllocation(plan.getResourceCalculator(),
             plan.getMinimumAllocation());
@@ -108,6 +108,8 @@ public class GreedyReservationAgent implements ReservationAgent {
     ReservationRequestInterpreter type = contract.getReservationRequests()
         .getInterpreter();
 
+    boolean hasGang = false;
+
     // Iterate the stages in backward from deadline
     for (ListIterator<ReservationRequest> li = 
         stages.listIterator(stages.size()); li.hasPrevious();) {
@@ -117,8 +119,10 @@ public class GreedyReservationAgent implements ReservationAgent {
       // validate the RR respect basic constraints
       validateInput(plan, currentReservationStage, totalCapacity);
 
+      hasGang |= currentReservationStage.getConcurrency() > 1;
+
       // run allocation for a single stage
-      Map<ReservationInterval, ReservationRequest> curAlloc =
+      Map<ReservationInterval, Resource> curAlloc =
           placeSingleStage(plan, tempAssigned, currentReservationStage,
               earliestStart, curDeadline, oldReservation, totalCapacity);
 
@@ -178,8 +182,7 @@ public class GreedyReservationAgent implements ReservationAgent {
 
     // create reservation with above allocations if not null/empty
 
-    ReservationRequest ZERO_RES =
-        ReservationRequest.newInstance(Resource.newInstance(0, 0), 0);
+    Resource ZERO_RES = Resource.newInstance(0, 0);
 
     long firstStartTime = findEarliestTime(allocations.keySet());
     
@@ -200,7 +203,7 @@ public class GreedyReservationAgent implements ReservationAgent {
         new InMemoryReservationAllocation(reservationId, contract, user,
             plan.getQueueName(), firstStartTime,
             findLatestTime(allocations.keySet()), allocations,
-            plan.getResourceCalculator(), plan.getMinimumAllocation());
+            plan.getResourceCalculator(), plan.getMinimumAllocation(), hasGang);
     if (oldReservation != null) {
       return plan.updateReservation(capReservation);
     } else {
@@ -242,13 +245,13 @@ public class GreedyReservationAgent implements ReservationAgent {
    * previous instant in time until the time-window is exhausted or we placed
    * all the user request.
    */
-  private Map<ReservationInterval, ReservationRequest> placeSingleStage(
+  private Map<ReservationInterval, Resource> placeSingleStage(
       Plan plan, RLESparseResourceAllocation tempAssigned,
       ReservationRequest rr, long earliestStart, long curDeadline,
       ReservationAllocation oldResAllocation, final Resource totalCapacity) {
 
-    Map<ReservationInterval, ReservationRequest> allocationRequests =
-        new HashMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> allocationRequests =
+        new HashMap<ReservationInterval, Resource>();
 
     // compute the gang as a resource and get the duration
     Resource gang = Resources.multiply(rr.getCapability(), rr.getConcurrency());
@@ -322,7 +325,7 @@ public class GreedyReservationAgent implements ReservationAgent {
 
         ReservationInterval reservationInt =
             new ReservationInterval(curDeadline - dur, curDeadline);
-        ReservationRequest reservationRes =
+        ReservationRequest reservationRequest =
             ReservationRequest.newInstance(rr.getCapability(),
                 rr.getConcurrency() * maxGang, rr.getConcurrency(),
                 rr.getDuration());
@@ -331,6 +334,8 @@ public class GreedyReservationAgent implements ReservationAgent {
         // placing other ReservationRequest within the same
         // ReservationDefinition,
         // and we must avoid double-counting the available resources
+        final Resource reservationRes = ReservationSystemUtil.toResource(
+            reservationRequest);
         tempAssigned.addInterval(reservationInt, reservationRes);
         allocationRequests.put(reservationInt, reservationRes);
 
@@ -350,7 +355,7 @@ public class GreedyReservationAgent implements ReservationAgent {
       // If we are here is becasue we did not manage to satisfy this request.
       // So we need to remove unwanted side-effect from tempAssigned (needed
       // for ANY).
-      for (Map.Entry<ReservationInterval, ReservationRequest> tempAllocation :
+      for (Map.Entry<ReservationInterval, Resource> tempAllocation :
         allocationRequests.entrySet()) {
         tempAssigned.removeInterval(tempAllocation.getKey(),
             tempAllocation.getValue());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
index ce2e7d7..50d66cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryPlan.java
@@ -31,7 +31,6 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.yarn.api.records.ReservationId;
-import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
@@ -110,7 +109,7 @@ class InMemoryPlan implements Plan {
 
   private void incrementAllocation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
-    Map<ReservationInterval, ReservationRequest> allocationRequests =
+    Map<ReservationInterval, Resource> allocationRequests =
         reservation.getAllocationRequests();
     // check if we have encountered the user earlier and if not add an entry
     String user = reservation.getUser();
@@ -119,7 +118,7 @@ class InMemoryPlan implements Plan {
       resAlloc = new RLESparseResourceAllocation(resCalc, minAlloc);
       userResourceAlloc.put(user, resAlloc);
     }
-    for (Map.Entry<ReservationInterval, ReservationRequest> r : allocationRequests
+    for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
       resAlloc.addInterval(r.getKey(), r.getValue());
       rleSparseVector.addInterval(r.getKey(), r.getValue());
@@ -128,11 +127,11 @@ class InMemoryPlan implements Plan {
 
   private void decrementAllocation(ReservationAllocation reservation) {
     assert (readWriteLock.isWriteLockedByCurrentThread());
-    Map<ReservationInterval, ReservationRequest> allocationRequests =
+    Map<ReservationInterval, Resource> allocationRequests =
         reservation.getAllocationRequests();
     String user = reservation.getUser();
     RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
-    for (Map.Entry<ReservationInterval, ReservationRequest> r : allocationRequests
+    for (Map.Entry<ReservationInterval, Resource> r : allocationRequests
         .entrySet()) {
       resAlloc.removeInterval(r.getKey(), r.getValue());
       rleSparseVector.removeInterval(r.getKey(), r.getValue());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
index fc8407b..a4dd23b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/InMemoryReservationAllocation.java
@@ -22,7 +22,6 @@ import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
-import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -40,7 +39,7 @@ class InMemoryReservationAllocation implements ReservationAllocation {
   private final ReservationDefinition contract;
   private final long startTime;
   private final long endTime;
-  private final Map<ReservationInterval, ReservationRequest> allocationRequests;
+  private final Map<ReservationInterval, Resource> allocationRequests;
   private boolean hasGang = false;
   private long acceptedAt = -1;
 
@@ -49,22 +48,29 @@ class InMemoryReservationAllocation implements ReservationAllocation {
   InMemoryReservationAllocation(ReservationId reservationID,
       ReservationDefinition contract, String user, String planName,
       long startTime, long endTime,
-      Map<ReservationInterval, ReservationRequest> allocationRequests,
+      Map<ReservationInterval, Resource> allocations,
       ResourceCalculator calculator, Resource minAlloc) {
+    this(reservationID, contract, user, planName, startTime, endTime,
+        allocations, calculator, minAlloc, false);
+  }
+
+  InMemoryReservationAllocation(ReservationId reservationID,
+      ReservationDefinition contract, String user, String planName,
+      long startTime, long endTime,
+      Map<ReservationInterval, Resource> allocations,
+      ResourceCalculator calculator, Resource minAlloc, boolean hasGang) {
     this.contract = contract;
     this.startTime = startTime;
     this.endTime = endTime;
     this.reservationID = reservationID;
     this.user = user;
-    this.allocationRequests = allocationRequests;
+    this.allocationRequests = allocations;
     this.planName = planName;
+    this.hasGang = hasGang;
     resourcesOverTime = new RLESparseResourceAllocation(calculator, minAlloc);
-    for (Map.Entry<ReservationInterval, ReservationRequest> r : allocationRequests
+    for (Map.Entry<ReservationInterval, Resource> r : allocations
         .entrySet()) {
       resourcesOverTime.addInterval(r.getKey(), r.getValue());
-      if (r.getValue().getConcurrency() > 1) {
-        hasGang = true;
-      }
     }
   }
 
@@ -89,7 +95,7 @@ class InMemoryReservationAllocation implements ReservationAllocation {
   }
 
   @Override
-  public Map<ReservationInterval, ReservationRequest> getAllocationRequests() {
+  public Map<ReservationInterval, Resource> getAllocationRequests() {
     return Collections.unmodifiableMap(allocationRequests);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
index 3f6f405..2957cc6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.NavigableMap;
@@ -31,9 +30,7 @@ import java.util.TreeMap;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
-import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -80,14 +77,11 @@ public class RLESparseResourceAllocation {
    * 
    * @param reservationInterval the interval for which the resource is to be
    *          added
-   * @param capacity the resource to be added
+   * @param totCap the resource to be added
    * @return true if addition is successful, false otherwise
    */
   public boolean addInterval(ReservationInterval reservationInterval,
-      ReservationRequest capacity) {
-    Resource totCap =
-        Resources.multiply(capacity.getCapability(),
-            (float) capacity.getNumContainers());
+      Resource totCap) {
     if (totCap.equals(ZERO_RESOURCE)) {
       return true;
     }
@@ -143,44 +137,15 @@ public class RLESparseResourceAllocation {
   }
 
   /**
-   * Add multiple resources for the specified interval
-   * 
-   * @param reservationInterval the interval for which the resource is to be
-   *          added
-   * @param ReservationRequests the resources to be added
-   * @param clusterResource the total resources in the cluster
-   * @return true if addition is successful, false otherwise
-   */
-  public boolean addCompositeInterval(ReservationInterval reservationInterval,
-      List<ReservationRequest> ReservationRequests, Resource clusterResource) {
-    ReservationRequest aggregateReservationRequest =
-        Records.newRecord(ReservationRequest.class);
-    Resource capacity = Resource.newInstance(0, 0);
-    for (ReservationRequest ReservationRequest : ReservationRequests) {
-      Resources.addTo(capacity, Resources.multiply(
-          ReservationRequest.getCapability(),
-          ReservationRequest.getNumContainers()));
-    }
-    aggregateReservationRequest.setNumContainers((int) Math.ceil(Resources
-        .divide(resourceCalculator, clusterResource, capacity, minAlloc)));
-    aggregateReservationRequest.setCapability(minAlloc);
-
-    return addInterval(reservationInterval, aggregateReservationRequest);
-  }
-
-  /**
    * Removes a resource for the specified interval
    * 
    * @param reservationInterval the interval for which the resource is to be
    *          removed
-   * @param capacity the resource to be removed
+   * @param totCap the resource to be removed
    * @return true if removal is successful, false otherwise
    */
   public boolean removeInterval(ReservationInterval reservationInterval,
-      ReservationRequest capacity) {
-    Resource totCap =
-        Resources.multiply(capacity.getCapability(),
-            (float) capacity.getNumContainers());
+      Resource totCap) {
     if (totCap.equals(ZERO_RESOURCE)) {
       return true;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
index 89c0e55..0d3c692 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAllocation.java
@@ -22,7 +22,6 @@ import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
-import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
 
 /**
@@ -71,7 +70,7 @@ public interface ReservationAllocation extends
    * @return the allocationRequests the map of resources requested against the
    *         time interval for which they were
    */
-  public Map<ReservationInterval, ReservationRequest> getAllocationRequests();
+  public Map<ReservationInterval, Resource> getAllocationRequests();
 
   /**
    * Return a string identifying the plan to which the reservation belongs

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemUtil.java
new file mode 100644
index 0000000..8affae4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemUtil.java
@@ -0,0 +1,51 @@
+/**
+ * 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.reservation;
+
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import java.util.HashMap;
+import java.util.Map;
+
+final class ReservationSystemUtil {
+
+  private ReservationSystemUtil() {
+    // not called
+  }
+
+  public static Resource toResource(ReservationRequest request) {
+    Resource resource = Resources.multiply(request.getCapability(),
+        (float) request.getNumContainers());
+    return resource;
+  }
+
+  public static Map<ReservationInterval, Resource> toResources(
+      Map<ReservationInterval, ReservationRequest> allocations) {
+    Map<ReservationInterval, Resource> resources =
+        new HashMap<ReservationInterval, Resource>();
+    for (Map.Entry<ReservationInterval, ReservationRequest> entry :
+        allocations.entrySet()) {
+      resources.put(entry.getKey(),
+          toResource(entry.getValue()));
+    }
+    return resources;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
index bfaf06b..be1d69a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
@@ -378,14 +378,15 @@ public class ReservationSystemTestUtil {
     return rr;
   }
 
-  public static Map<ReservationInterval, ReservationRequest> generateAllocation(
+  public static Map<ReservationInterval, Resource> generateAllocation(
       long startTime, long step, int[] alloc) {
-    Map<ReservationInterval, ReservationRequest> req =
-        new TreeMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> req =
+        new TreeMap<ReservationInterval, Resource>();
     for (int i = 0; i < alloc.length; i++) {
       req.put(new ReservationInterval(startTime + i * step, startTime + (i + 1)
-          * step), ReservationRequest.newInstance(
-          Resource.newInstance(1024, 1), alloc[i]));
+          * step), ReservationSystemUtil.toResource(ReservationRequest
+          .newInstance(
+          Resource.newInstance(1024, 1), alloc[i])));
     }
     return req;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java
index 61561e9..19f876d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.Map;
@@ -198,12 +197,14 @@ public class TestCapacityOverTimePolicy {
   @Test(expected = PlanningQuotaException.class)
   public void testFailAvg() throws IOException, PlanningException {
     // generate an allocation which violates the 25% average single-shot
-    Map<ReservationInterval, ReservationRequest> req =
-        new TreeMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> req =
+        new TreeMap<ReservationInterval, Resource>();
     long win = timeWindow / 2 + 100;
     int cont = (int) Math.ceil(0.5 * totCont);
     req.put(new ReservationInterval(initTime, initTime + win),
-        ReservationRequest.newInstance(Resource.newInstance(1024, 1), cont));
+        ReservationSystemUtil.toResource(
+            ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+                cont)));
 
     assertTrue(plan.toString(),
         plan.addReservation(new InMemoryReservationAllocation(
@@ -214,12 +215,13 @@ public class TestCapacityOverTimePolicy {
   @Test
   public void testFailAvgBySum() throws IOException, PlanningException {
     // generate an allocation which violates the 25% average by sum
-    Map<ReservationInterval, ReservationRequest> req =
-        new TreeMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> req =
+        new TreeMap<ReservationInterval, Resource>();
     long win = 86400000 / 4 + 1;
     int cont = (int) Math.ceil(0.5 * totCont);
     req.put(new ReservationInterval(initTime, initTime + win),
-        ReservationRequest.newInstance(Resource.newInstance(1024, 1), cont));
+        ReservationSystemUtil.toResource(ReservationRequest.newInstance(Resource
+            .newInstance(1024, 1), cont)));
     assertTrue(plan.toString(),
         plan.addReservation(new InMemoryReservationAllocation(
             ReservationSystemTestUtil.getNewReservationId(), null, "u1",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestGreedyReservationAgent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestGreedyReservationAgent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestGreedyReservationAgent.java
index b8cf6c5..de94dcd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestGreedyReservationAgent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestGreedyReservationAgent.java
@@ -516,7 +516,7 @@ public class TestGreedyReservationAgent {
                 .generateAllocation(0, step, f), res, minAlloc)));
 
     int[] f2 = { 5, 5, 5, 5, 5, 5, 5 };
-    Map<ReservationInterval, ReservationRequest> alloc = 
+    Map<ReservationInterval, Resource> alloc =
         ReservationSystemTestUtil.generateAllocation(5000, step, f2);
     assertTrue(plan.toString(),
         plan.addReservation(new InMemoryReservationAllocation(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
index 91c1962..722fb29 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
@@ -100,9 +100,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length,
             alloc.length, allocations.values());
+    Map<ReservationInterval, Resource> allocs =
+        ReservationSystemUtil.toResources(allocations);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length, allocations, resCalc, minAlloc);
+            start, start + alloc.length, allocs, resCalc, minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation);
@@ -132,9 +134,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length,
             alloc.length, allocations.values());
+    Map<ReservationInterval, Resource> allocs = ReservationSystemUtil.toResources
+        (allocations);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length, allocations, resCalc, minAlloc);
+            start, start + alloc.length, allocs, resCalc, minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation);
@@ -158,9 +162,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length,
             alloc.length, allocations.values());
+    Map<ReservationInterval, Resource> allocs = ReservationSystemUtil.toResources
+        (allocations);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length, allocations, resCalc, minAlloc);
+            start, start + alloc.length, allocs, resCalc, minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation);
@@ -202,9 +208,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length,
             alloc.length, allocations.values());
+    Map<ReservationInterval, Resource> allocs = ReservationSystemUtil.toResources
+        (allocations);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length, allocations, resCalc, minAlloc);
+            start, start + alloc.length, allocs, resCalc, minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation);
@@ -226,9 +234,12 @@ public class TestInMemoryPlan {
     rDef =
         createSimpleReservationDefinition(start, start + updatedAlloc.length,
             updatedAlloc.length, allocations.values());
+    Map<ReservationInterval, Resource> updatedAllocs =
+        ReservationSystemUtil.toResources(allocations);
     rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + updatedAlloc.length, allocations, resCalc, minAlloc);
+            start, start + updatedAlloc.length, updatedAllocs, resCalc,
+            minAlloc);
     try {
       plan.updateReservation(rAllocation);
     } catch (PlanningException e) {
@@ -260,9 +271,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length,
             alloc.length, allocations.values());
+    Map<ReservationInterval, Resource> allocs =
+        ReservationSystemUtil.toResources(allocations);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length, allocations, resCalc, minAlloc);
+            start, start + alloc.length, allocs, resCalc, minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.updateReservation(rAllocation);
@@ -290,9 +303,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length,
             alloc.length, allocations.values());
+    Map<ReservationInterval, Resource> allocs =
+        ReservationSystemUtil.toResources(allocations);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length, allocations, resCalc, minAlloc);
+            start, start + alloc.length, allocs, resCalc, minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation);
@@ -359,9 +374,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef1 =
         createSimpleReservationDefinition(start, start + alloc1.length,
             alloc1.length, allocations1.values());
+    Map<ReservationInterval, Resource> allocs1 =
+        ReservationSystemUtil.toResources(allocations1);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID1, rDef1, user,
-            planName, start, start + alloc1.length, allocations1, resCalc,
+            planName, start, start + alloc1.length, allocs1, resCalc,
             minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID1));
     try {
@@ -388,9 +405,11 @@ public class TestInMemoryPlan {
     ReservationDefinition rDef2 =
         createSimpleReservationDefinition(start, start + alloc2.length,
             alloc2.length, allocations2.values());
+    Map<ReservationInterval, Resource> allocs2 =
+        ReservationSystemUtil.toResources(allocations2);
     rAllocation =
         new InMemoryReservationAllocation(reservationID2, rDef2, user,
-            planName, start, start + alloc2.length, allocations2, resCalc,
+            planName, start, start + alloc2.length, allocs2, resCalc,
             minAlloc);
     Assert.assertNull(plan.getReservationById(reservationID2));
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryReservationAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryReservationAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryReservationAllocation.java
index 76f39dc..55224a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryReservationAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryReservationAllocation.java
@@ -69,7 +69,7 @@ public class TestInMemoryReservationAllocation {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length + 1,
             alloc.length);
-    Map<ReservationInterval, ReservationRequest> allocations =
+    Map<ReservationInterval, Resource> allocations =
         generateAllocation(start, alloc, false, false);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
@@ -91,7 +91,7 @@ public class TestInMemoryReservationAllocation {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length + 1,
             alloc.length);
-    Map<ReservationInterval, ReservationRequest> allocations =
+    Map<ReservationInterval, Resource> allocations =
         generateAllocation(start, alloc, true, false);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
@@ -114,7 +114,7 @@ public class TestInMemoryReservationAllocation {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length + 1,
             alloc.length);
-    Map<ReservationInterval, ReservationRequest> allocations =
+    Map<ReservationInterval, Resource> allocations =
         generateAllocation(start, alloc, true, false);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
@@ -137,8 +137,8 @@ public class TestInMemoryReservationAllocation {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length + 1,
             alloc.length);
-    Map<ReservationInterval, ReservationRequest> allocations =
-        new HashMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> allocations =
+        new HashMap<ReservationInterval, Resource>();
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
             start, start + alloc.length + 1, allocations, resCalc, minAlloc);
@@ -156,11 +156,13 @@ public class TestInMemoryReservationAllocation {
     ReservationDefinition rDef =
         createSimpleReservationDefinition(start, start + alloc.length + 1,
             alloc.length);
-    Map<ReservationInterval, ReservationRequest> allocations =
-        generateAllocation(start, alloc, false, true);
+    boolean isGang = true;
+    Map<ReservationInterval, Resource> allocations =
+        generateAllocation(start, alloc, false, isGang);
     ReservationAllocation rAllocation =
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
-            start, start + alloc.length + 1, allocations, resCalc, minAlloc);
+            start, start + alloc.length + 1, allocations, resCalc, minAlloc,
+            isGang);
     doAssertions(rAllocation, reservationID, rDef, allocations, start, alloc);
     Assert.assertTrue(rAllocation.containsGangs());
     for (int i = 0; i < alloc.length; i++) {
@@ -171,7 +173,7 @@ public class TestInMemoryReservationAllocation {
 
   private void doAssertions(ReservationAllocation rAllocation,
       ReservationId reservationID, ReservationDefinition rDef,
-      Map<ReservationInterval, ReservationRequest> allocations, int start,
+      Map<ReservationInterval, Resource> allocations, int start,
       int[] alloc) {
     Assert.assertEquals(reservationID, rAllocation.getReservationId());
     Assert.assertEquals(rDef, rAllocation.getReservationDefinition());
@@ -198,10 +200,10 @@ public class TestInMemoryReservationAllocation {
     return rDef;
   }
 
-  private Map<ReservationInterval, ReservationRequest> generateAllocation(
+  private Map<ReservationInterval, Resource> generateAllocation(
       int startTime, int[] alloc, boolean isStep, boolean isGang) {
-    Map<ReservationInterval, ReservationRequest> req =
-        new HashMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> req =
+        new HashMap<ReservationInterval, Resource>();
     int numContainers = 0;
     for (int i = 0; i < alloc.length; i++) {
       if (isStep) {
@@ -215,7 +217,8 @@ public class TestInMemoryReservationAllocation {
       if (isGang) {
         rr.setConcurrency(numContainers);
       }
-      req.put(new ReservationInterval(startTime + i, startTime + i + 1), rr);
+      req.put(new ReservationInterval(startTime + i, startTime + i + 1),
+          ReservationSystemUtil.toResource(rr));
     }
     return req;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
index c7301c7..d0f4dc6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
@@ -46,9 +46,9 @@ public class TestRLESparseResourceAllocation {
         new RLESparseResourceAllocation(resCalc, minAlloc);
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    Set<Entry<ReservationInterval, ReservationRequest>> inputs =
+    Set<Entry<ReservationInterval, Resource>> inputs =
         generateAllocation(start, alloc, false).entrySet();
-    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+    for (Entry<ReservationInterval, Resource> ip : inputs) {
       rleSparseVector.addInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
@@ -63,7 +63,7 @@ public class TestRLESparseResourceAllocation {
     }
     Assert.assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
-    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+    for (Entry<ReservationInterval, Resource> ip : inputs) {
       rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
@@ -83,9 +83,9 @@ public class TestRLESparseResourceAllocation {
         new RLESparseResourceAllocation(resCalc, minAlloc);
     int[] alloc = { 10, 10, 10, 10, 10, 10 };
     int start = 100;
-    Set<Entry<ReservationInterval, ReservationRequest>> inputs =
+    Set<Entry<ReservationInterval, Resource>> inputs =
         generateAllocation(start, alloc, true).entrySet();
-    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+    for (Entry<ReservationInterval, Resource> ip : inputs) {
       rleSparseVector.addInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
@@ -101,8 +101,8 @@ public class TestRLESparseResourceAllocation {
     }
     Assert.assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
-    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
-      rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
+    for (Entry<ReservationInterval, Resource> ip : inputs) {
+      rleSparseVector.removeInterval(ip.getKey(),ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
     for (int i = 0; i < alloc.length; i++) {
@@ -121,9 +121,9 @@ public class TestRLESparseResourceAllocation {
         new RLESparseResourceAllocation(resCalc, minAlloc);
     int[] alloc = { 0, 5, 10, 10, 5, 0 };
     int start = 100;
-    Set<Entry<ReservationInterval, ReservationRequest>> inputs =
+    Set<Entry<ReservationInterval, Resource>> inputs =
         generateAllocation(start, alloc, true).entrySet();
-    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+    for (Entry<ReservationInterval, Resource> ip : inputs) {
       rleSparseVector.addInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
@@ -139,7 +139,7 @@ public class TestRLESparseResourceAllocation {
     }
     Assert.assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
-    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+    for (Entry<ReservationInterval, Resource> ip : inputs) {
       rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
@@ -157,17 +157,17 @@ public class TestRLESparseResourceAllocation {
     RLESparseResourceAllocation rleSparseVector =
         new RLESparseResourceAllocation(resCalc, minAlloc);
     rleSparseVector.addInterval(new ReservationInterval(0, Long.MAX_VALUE),
-        ReservationRequest.newInstance(Resource.newInstance(0, 0), (0)));
+        Resource.newInstance(0, 0));
     LOG.info(rleSparseVector.toString());
     Assert.assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(new Random().nextLong()));
     Assert.assertTrue(rleSparseVector.isEmpty());
   }
 
-  private Map<ReservationInterval, ReservationRequest> generateAllocation(
+  private Map<ReservationInterval, Resource> generateAllocation(
       int startTime, int[] alloc, boolean isStep) {
-    Map<ReservationInterval, ReservationRequest> req =
-        new HashMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> req =
+        new HashMap<ReservationInterval, Resource>();
     int numContainers = 0;
     for (int i = 0; i < alloc.length; i++) {
       if (isStep) {
@@ -176,9 +176,8 @@ public class TestRLESparseResourceAllocation {
         numContainers = alloc[i];
       }
       req.put(new ReservationInterval(startTime + i, startTime + i + 1),
-
-      ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-          (numContainers)));
+          ReservationSystemUtil.toResource(ReservationRequest.newInstance(
+              Resource.newInstance(1024, 1), (numContainers))));
     }
     return req;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb8b2c18/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSimpleCapacityReplanner.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSimpleCapacityReplanner.java
index 1ca9f2e..d4a97ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSimpleCapacityReplanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSimpleCapacityReplanner.java
@@ -146,14 +146,15 @@ public class TestSimpleCapacityReplanner {
     }
   }
 
-  private Map<ReservationInterval, ReservationRequest> generateAllocation(
+  private Map<ReservationInterval, Resource> generateAllocation(
       int startTime, int[] alloc) {
-    Map<ReservationInterval, ReservationRequest> req =
-        new TreeMap<ReservationInterval, ReservationRequest>();
+    Map<ReservationInterval, Resource> req =
+        new TreeMap<ReservationInterval, Resource>();
     for (int i = 0; i < alloc.length; i++) {
       req.put(new ReservationInterval(startTime + i, startTime + i + 1),
-          ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-              alloc[i]));
+          ReservationSystemUtil.toResource(
+              ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+                  alloc[i])));
     }
     return req;
   }