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 cd...@apache.org on 2014/10/04 02:09:59 UTC

[01/16] git commit: YARN-1707. Introduce APIs to add/remove/resize queues in the CapacityScheduler. Contributed by Carlo Curino and Subru Krishnan (cherry picked from commit aac47fda7fecda9fc18ade34d633eca895865a70)

Repository: hadoop
Updated Branches:
  refs/heads/trunk 7f6ed7fe3 -> c8212bacb


YARN-1707. Introduce APIs to add/remove/resize queues in the CapacityScheduler. Contributed by Carlo Curino and Subru Krishnan
(cherry picked from commit aac47fda7fecda9fc18ade34d633eca895865a70)

Conflicts:
	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


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

Branch: refs/heads/trunk
Commit: eb3e40b833b9d82c1556843f960194dc42e482f3
Parents: 7f6ed7f
Author: carlo curino <Carlo Curino>
Authored: Fri Sep 12 16:43:38 2014 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Oct 3 15:41:02 2014 -0700

----------------------------------------------------------------------
 .../scheduler/AbstractYarnScheduler.java        |  25 ++
 .../SchedulerDynamicEditException.java          |  13 +
 .../scheduler/YarnScheduler.java                |  42 +++
 .../scheduler/capacity/CapacityScheduler.java   | 209 ++++++++++++--
 .../scheduler/capacity/LeafQueue.java           |  28 +-
 .../scheduler/capacity/ParentQueue.java         |   8 +-
 .../scheduler/capacity/PlanQueue.java           | 193 +++++++++++++
 .../scheduler/capacity/ReservationQueue.java    |  98 +++++++
 .../scheduler/common/QueueEntitlement.java      |  28 ++
 .../webapp/dao/CapacitySchedulerQueueInfo.java  |   9 +
 .../capacity/TestCapacityScheduler.java         |   2 +-
 .../TestCapacitySchedulerDynamicBehavior.java   | 282 +++++++++++++++++++
 .../capacity/TestReservationQueue.java          | 103 +++++++
 13 files changed, 1012 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb3e40b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.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/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index 0b5447b..8e8d627 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMoveEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
@@ -220,6 +221,24 @@ public abstract class AbstractYarnScheduler
         + " does not support moving apps between queues");
   }
 
+  public void removeQueue(String queueName) throws YarnException {
+    throw new YarnException(getClass().getSimpleName()
+        + " does not support removing queues");
+  }
+
+  @Override
+  public void addQueue(Queue newQueue) throws YarnException {
+    throw new YarnException(getClass().getSimpleName()
+        + " does not support this operation");
+  }
+
+  @Override
+  public void setEntitlement(String queue, QueueEntitlement entitlement)
+      throws YarnException {
+    throw new YarnException(getClass().getSimpleName()
+        + " does not support this operation");
+  }
+
   private void killOrphanContainerOnNode(RMNode node,
       NMContainerStatus container) {
     if (!container.getContainerState().equals(ContainerState.COMPLETE)) {
@@ -503,4 +522,10 @@ public abstract class AbstractYarnScheduler
   public EnumSet<SchedulerResourceTypes> getSchedulingResourceTypes() {
     return EnumSet.of(SchedulerResourceTypes.MEMORY);
   }
+
+  @Override
+  public Set<String> getPlanQueues() throws YarnException {
+    throw new YarnException(getClass().getSimpleName()
+        + " does not support reservations");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb3e40b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerDynamicEditException.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/scheduler/SchedulerDynamicEditException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerDynamicEditException.java
new file mode 100644
index 0000000..42dc36a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerDynamicEditException.java
@@ -0,0 +1,13 @@
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+public class SchedulerDynamicEditException extends YarnException {
+
+  private static final long serialVersionUID = 7100374511387193257L;
+
+  public SchedulerDynamicEditException(String string) {
+    super(string);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb3e40b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.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/scheduler/YarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
index b6c1018..d1b5275 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 import java.io.IOException;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -41,6 +42,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
 
@@ -224,6 +226,46 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
   void killAllAppsInQueue(String queueName) throws YarnException;
 
   /**
+   * Remove an existing queue. Implementations might limit when a queue could be
+   * removed (e.g., must have zero entitlement, and no applications running, or
+   * must be a leaf, etc..).
+   *
+   * @param queueName name of the queue to remove
+   * @throws YarnException
+   */
+  void removeQueue(String queueName) throws YarnException;
+
+  /**
+   * Add to the scheduler a new Queue. Implementations might limit what type of
+   * queues can be dynamically added (e.g., Queue must be a leaf, must be
+   * attached to existing parent, must have zero entitlement).
+   *
+   * @param newQueue the queue being added.
+   * @throws YarnException
+   */
+  void addQueue(Queue newQueue) throws YarnException;
+
+  /**
+   * This method increase the entitlement for current queue (must respect
+   * invariants, e.g., no overcommit of parents, non negative, etc.).
+   * Entitlement is a general term for weights in FairScheduler, capacity for
+   * the CapacityScheduler, etc.
+   *
+   * @param queue the queue for which we change entitlement
+   * @param entitlement the new entitlement for the queue (capacity,
+   *              maxCapacity, etc..)
+   * @throws YarnException
+   */
+  void setEntitlement(String queue, QueueEntitlement entitlement)
+      throws YarnException;
+
+  /**
+   * Gets the list of names for queues managed by the Reservation System
+   * @return the list of queues which support reservations
+   */
+  public Set<String> getPlanQueues() throws YarnException;  
+
+  /**
    * Return a collection of the resource types that are considered when
    * scheduling
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb3e40b8/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
----------------------------------------------------------------------
diff --git a/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 b/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
index e0816b5..ff6db3a 100644
--- a/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
+++ b/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
@@ -24,6 +24,8 @@ import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.HashSet;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -90,6 +92,11 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
+
 @LimitedPrivate("yarn")
 @Evolving
 @SuppressWarnings("unchecked")
@@ -473,9 +480,12 @@ public class CapacityScheduler extends
   private void validateExistingQueues(
       Map<String, CSQueue> queues, Map<String, CSQueue> newQueues) 
   throws IOException {
-    for (String queue : queues.keySet()) {
-      if (!newQueues.containsKey(queue)) {
-        throw new IOException(queue + " cannot be found during refresh!");
+    // check that all static queues are included in the newQueues list
+    for (Map.Entry<String, CSQueue> e : queues.entrySet()) {
+      if (!(e.getValue() instanceof ReservationQueue)) {
+        if (!newQueues.containsKey(e.getKey())) {
+          throw new IOException(e.getKey() + " cannot be found during refresh!");
+        }
       }
     }
   }
@@ -507,26 +517,42 @@ public class CapacityScheduler extends
       Map<String, CSQueue> oldQueues, 
       QueueHook hook) throws IOException {
     CSQueue queue;
+    String fullQueueName =
+        (parent == null) ? queueName
+            : (parent.getQueuePath() + "." + queueName);
     String[] childQueueNames = 
-      conf.getQueues((parent == null) ? 
-          queueName : (parent.getQueuePath()+"."+queueName));
+      conf.getQueues(fullQueueName);
+    boolean isReservableQueue = conf.isReservableQueue(fullQueueName);
     if (childQueueNames == null || childQueueNames.length == 0) {
       if (null == parent) {
         throw new IllegalStateException(
             "Queue configuration missing child queue names for " + queueName);
       }
-      queue = 
-          new LeafQueue(csContext, queueName, parent, oldQueues.get(queueName));
-      
-      // Used only for unit tests
-      queue = hook.hook(queue);
+      // Check if the queue will be dynamically managed by the Reservation
+      // system
+      if (isReservableQueue) {
+        queue =
+            new PlanQueue(csContext, queueName, parent,
+                oldQueues.get(queueName));
+      } else {
+        queue =
+            new LeafQueue(csContext, queueName, parent,
+                oldQueues.get(queueName));
+
+        // Used only for unit tests
+        queue = hook.hook(queue);
+      }
     } else {
+      if (isReservableQueue) {
+        throw new IllegalStateException(
+            "Only Leaf Queues can be reservable for " + queueName);
+      }
       ParentQueue parentQueue = 
         new ParentQueue(csContext, queueName, parent, oldQueues.get(queueName));
 
       // Used only for unit tests
       queue = hook.hook(parentQueue);
-      
+
       List<CSQueue> childQueues = new ArrayList<CSQueue>();
       for (String childQueueName : childQueueNames) {
         CSQueue childQueue = 
@@ -548,7 +574,7 @@ public class CapacityScheduler extends
     return queue;
   }
 
-  synchronized CSQueue getQueue(String queueName) {
+  public synchronized CSQueue getQueue(String queueName) {
     if (queueName == null) {
       return null;
     }
@@ -716,7 +742,7 @@ public class CapacityScheduler extends
       ApplicationAttemptId applicationAttemptId,
       RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
     LOG.info("Application Attempt " + applicationAttemptId + " is done." +
-    		" finalState=" + rmAppAttemptFinalState);
+        " finalState=" + rmAppAttemptFinalState);
     
     FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId);
     SchedulerApplication<FiCaSchedulerApp> application =
@@ -996,9 +1022,16 @@ public class CapacityScheduler extends
     case APP_ADDED:
     {
       AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event;
-      addApplication(appAddedEvent.getApplicationId(),
-        appAddedEvent.getQueue(),
-        appAddedEvent.getUser(), appAddedEvent.getIsAppRecovering());
+      String queueName =
+          resolveReservationQueueName(appAddedEvent.getQueue(),
+              appAddedEvent.getApplicationId(),
+              appAddedEvent.getReservationID());
+      if (queueName != null) {
+        addApplication(appAddedEvent.getApplicationId(),
+            queueName,
+            appAddedEvent.getUser(),
+            appAddedEvent.getIsAppRecovering());
+      }
     }
     break;
     case APP_REMOVED:
@@ -1231,6 +1264,123 @@ public class CapacityScheduler extends
     }
   }
 
+  private synchronized String resolveReservationQueueName(String queueName,
+      ApplicationId applicationId, ReservationId reservationID) {
+    CSQueue queue = getQueue(queueName);
+    // Check if the queue is a plan queue
+    if ((queue == null) || !(queue instanceof PlanQueue)) {
+      return queueName;
+    }
+    if (reservationID != null) {
+      String resQName = reservationID.toString();
+      queue = getQueue(resQName);
+      if (queue == null) {
+        String message =
+            "Application "
+                + applicationId
+                + " submitted to a reservation which is not yet currently active: "
+                + resQName;
+        this.rmContext.getDispatcher().getEventHandler()
+            .handle(new RMAppRejectedEvent(applicationId, message));
+        return null;
+      }
+      // use the reservation queue to run the app
+      queueName = resQName;
+    } else {
+      // use the default child queue of the plan for unreserved apps
+      queueName = queueName + PlanQueue.DEFAULT_QUEUE_SUFFIX;
+    }
+    return queueName;
+  }
+
+  @Override
+  public synchronized void removeQueue(String queueName)
+      throws SchedulerDynamicEditException {
+    LOG.info("Removing queue: " + queueName);
+    CSQueue q = this.getQueue(queueName);
+    if (!(q instanceof ReservationQueue)) {
+      throw new SchedulerDynamicEditException("The queue that we are asked "
+          + "to remove (" + queueName + ") is not a ReservationQueue");
+    }
+    ReservationQueue disposableLeafQueue = (ReservationQueue) q;
+    // at this point we should have no more apps
+    if (disposableLeafQueue.getNumApplications() > 0) {
+      throw new SchedulerDynamicEditException("The queue " + queueName
+          + " is not empty " + disposableLeafQueue.getApplications().size()
+          + " active apps " + disposableLeafQueue.pendingApplications.size()
+          + " pending apps");
+    }
+
+    ((PlanQueue) disposableLeafQueue.getParent()).removeChildQueue(q);
+    this.queues.remove(queueName);
+    LOG.info("Removal of ReservationQueue " + queueName + " has succeeded");
+  }
+
+  @Override
+  public synchronized void addQueue(Queue queue)
+      throws SchedulerDynamicEditException {
+
+    if (!(queue instanceof ReservationQueue)) {
+      throw new SchedulerDynamicEditException("Queue " + queue.getQueueName()
+          + " is not a ReservationQueue");
+    }
+
+    ReservationQueue newQueue = (ReservationQueue) queue;
+
+    if (newQueue.getParent() == null
+        || !(newQueue.getParent() instanceof PlanQueue)) {
+      throw new SchedulerDynamicEditException("ParentQueue for "
+          + newQueue.getQueueName()
+          + " is not properly set (should be set and be a PlanQueue)");
+    }
+
+    PlanQueue parentPlan = (PlanQueue) newQueue.getParent();
+    String queuename = newQueue.getQueueName();
+    parentPlan.addChildQueue(newQueue);
+    this.queues.put(queuename, newQueue);
+    LOG.info("Creation of ReservationQueue " + newQueue + " succeeded");
+  }
+
+  @Override
+  public synchronized void setEntitlement(String inQueue,
+      QueueEntitlement entitlement) throws SchedulerDynamicEditException,
+      YarnException {
+    LeafQueue queue = getAndCheckLeafQueue(inQueue);
+    ParentQueue parent = (ParentQueue) queue.getParent();
+
+    if (!(queue instanceof ReservationQueue)) {
+      throw new SchedulerDynamicEditException("Entitlement can not be"
+          + " modified dynamically since queue " + inQueue
+          + " is not a ReservationQueue");
+    }
+
+    if (!(parent instanceof PlanQueue)) {
+      throw new SchedulerDynamicEditException("The parent of ReservationQueue "
+          + inQueue + " must be an PlanQueue");
+    }
+
+    ReservationQueue newQueue = (ReservationQueue) queue;
+
+    float sumChilds = ((PlanQueue) parent).sumOfChildCapacities();
+    float newChildCap = sumChilds - queue.getCapacity() + entitlement.getCapacity();
+
+    if (newChildCap >= 0 && newChildCap < 1.0f + CSQueueUtils.EPSILON) {
+      // note: epsilon checks here are not ok, as the epsilons might accumulate
+      // and become a problem in aggregate
+      if (Math.abs(entitlement.getCapacity() - queue.getCapacity()) == 0
+          && Math.abs(entitlement.getMaxCapacity() - queue.getMaximumCapacity()) == 0) {
+        return;
+      }
+      newQueue.setEntitlement(entitlement);
+    } else {
+      throw new SchedulerDynamicEditException(
+          "Sum of child queues would exceed 100% for PlanQueue: "
+              + parent.getQueueName());
+    }
+    LOG.info("Set entitlement for ReservationQueue " + inQueue + "  to "
+        + queue.getCapacity() + " request was (" + entitlement.getCapacity() + ")");
+  }
+
   @Override
   public synchronized String moveApplication(ApplicationId appId,
       String targetQueueName) throws YarnException {
@@ -1238,11 +1388,12 @@ public class CapacityScheduler extends
         getApplicationAttempt(ApplicationAttemptId.newInstance(appId, 0));
     String sourceQueueName = app.getQueue().getQueueName();
     LeafQueue source = getAndCheckLeafQueue(sourceQueueName);
-    LeafQueue dest = getAndCheckLeafQueue(targetQueueName);
+    String destQueueName = handleMoveToPlanQueue(targetQueueName);
+    LeafQueue dest = getAndCheckLeafQueue(destQueueName);
     // Validation check - ACLs, submission limits for user & queue
     String user = app.getUser();
     try {
-      dest.submitApplication(appId, user, targetQueueName);
+      dest.submitApplication(appId, user, destQueueName);
     } catch (AccessControlException e) {
       throw new YarnException(e);
     }
@@ -1261,7 +1412,7 @@ public class CapacityScheduler extends
     dest.submitApplicationAttempt(app, user);
     applications.get(appId).setQueue(dest);
     LOG.info("App: " + app.getApplicationId() + " successfully moved from "
-        + sourceQueueName + " to: " + targetQueueName);
+        + sourceQueueName + " to: " + destQueueName);
     return targetQueueName;
   }
 
@@ -1296,4 +1447,24 @@ public class CapacityScheduler extends
     return EnumSet
       .of(SchedulerResourceTypes.MEMORY, SchedulerResourceTypes.CPU);
   }
+  
+  private String handleMoveToPlanQueue(String targetQueueName) {
+    CSQueue dest = getQueue(targetQueueName);
+    if (dest != null && dest instanceof PlanQueue) {
+      // use the default child reservation queue of the plan
+      targetQueueName = targetQueueName + PlanQueue.DEFAULT_QUEUE_SUFFIX;
+    }
+    return targetQueueName;
+  }
+
+  @Override
+  public Set<String> getPlanQueues() {
+    Set<String> ret = new HashSet<String>();
+    for (Map.Entry<String, CSQueue> l : queues.entrySet()) {
+      if (l.getValue() instanceof PlanQueue) {
+        ret.add(l.getKey());
+      }
+    }
+    return ret;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb3e40b8/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
----------------------------------------------------------------------
diff --git a/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 b/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
index cdb6553..b9f5d5f 100644
--- a/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
+++ b/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
@@ -86,8 +86,8 @@ public class LeafQueue implements CSQueue {
   private int userLimit;
   private float userLimitFactor;
 
-  private int maxApplications;
-  private int maxApplicationsPerUser;
+  protected int maxApplications;
+  protected int maxApplicationsPerUser;
   
   private float maxAMResourcePerQueuePercent;
   private int maxActiveApplications; // Based on absolute max capacity
@@ -153,8 +153,7 @@ public class LeafQueue implements CSQueue {
             Resources.subtract(maximumAllocation, minimumAllocation), 
             maximumAllocation);
 
-    float capacity = 
-      (float)cs.getConfiguration().getCapacity(getQueuePath()) / 100;
+    float capacity = getCapacityFromConf();
     float absoluteCapacity = parent.getAbsoluteCapacity() * capacity;
 
     float maximumCapacity = 
@@ -221,6 +220,11 @@ public class LeafQueue implements CSQueue {
     this.activeApplications = new TreeSet<FiCaSchedulerApp>(applicationComparator);
   }
 
+  // externalizing in method, to allow overriding
+  protected float getCapacityFromConf() {
+    return (float)scheduler.getConfiguration().getCapacity(getQueuePath()) / 100;
+  }
+
   private synchronized void setupQueueConfigs(
       Resource clusterResource,
       float capacity, float absoluteCapacity, 
@@ -483,7 +487,7 @@ public class LeafQueue implements CSQueue {
    * Set user limit factor - used only for testing.
    * @param userLimitFactor new user limit factor
    */
-  synchronized void setUserLimitFactor(int userLimitFactor) {
+  synchronized void setUserLimitFactor(float userLimitFactor) {
     this.userLimitFactor = userLimitFactor;
   }
 
@@ -831,7 +835,7 @@ public class LeafQueue implements CSQueue {
           getApplication(reservedContainer.getApplicationAttemptId());
       synchronized (application) {
         return assignReservedContainer(application, node, reservedContainer,
-          clusterResource);
+            clusterResource);
       }
     }
     
@@ -1880,4 +1884,16 @@ public class LeafQueue implements CSQueue {
       getParent().detachContainer(clusterResource, application, rmContainer);
     }
   }
+
+  public void setCapacity(float capacity) {
+    this.capacity = capacity;
+  }
+
+  public void setAbsoluteCapacity(float absoluteCapacity) {
+    this.absoluteCapacity = absoluteCapacity;
+  }
+
+  public void setMaxApplications(int maxApplications) {
+    this.maxApplications = maxApplications;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb3e40b8/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
----------------------------------------------------------------------
diff --git a/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 b/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
index aa74be1..011c99c 100644
--- a/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
+++ b/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
@@ -75,7 +75,7 @@ public class ParentQueue implements CSQueue {
 
   private float usedCapacity = 0.0f;
 
-  private final Set<CSQueue> childQueues;
+  protected final Set<CSQueue> childQueues;
   private final Comparator<CSQueue> queueComparator;
   
   private Resource usedResources = Resources.createResource(0, 0);
@@ -159,7 +159,7 @@ public class ParentQueue implements CSQueue {
         ", fullname=" + getQueuePath()); 
   }
 
-  private synchronized void setupQueueConfigs(
+  protected synchronized void setupQueueConfigs(
       Resource clusterResource,
       float capacity, float absoluteCapacity, 
       float maximumCapacity, float absoluteMaxCapacity,
@@ -881,4 +881,8 @@ public class ParentQueue implements CSQueue {
       }
     }
   }
+
+  public Map<QueueACL, AccessControlList> getACLs() {
+    return acls;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb3e40b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java
new file mode 100644
index 0000000..4ada778
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java
@@ -0,0 +1,193 @@
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This represents a dynamic queue managed by the {@link ReservationSystem}.
+ * From the user perspective this is equivalent to a LeafQueue that respect
+ * reservations, but functionality wise is a sub-class of ParentQueue
+ *
+ */
+public class PlanQueue extends ParentQueue {
+
+  public static final String DEFAULT_QUEUE_SUFFIX = "-default";
+
+  private static final Logger LOG = LoggerFactory.getLogger(PlanQueue.class);
+
+  private int maxAppsForReservation;
+  private int maxAppsPerUserForReservation;
+  private int userLimit;
+  private float userLimitFactor;
+  protected CapacitySchedulerContext schedulerContext;
+  private boolean showReservationsAsQueues;
+
+  public PlanQueue(CapacitySchedulerContext cs, String queueName,
+      CSQueue parent, CSQueue old) {
+    super(cs, queueName, parent, old);
+
+    this.schedulerContext = cs;
+    // Set the reservation queue attributes for the Plan
+    CapacitySchedulerConfiguration conf = cs.getConfiguration();
+    String queuePath = super.getQueuePath();
+    int maxAppsForReservation = conf.getMaximumApplicationsPerQueue(queuePath);
+    showReservationsAsQueues = conf.getShowReservationAsQueues(queuePath);
+    if (maxAppsForReservation < 0) {
+      maxAppsForReservation =
+          (int) (CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS * super
+              .getAbsoluteCapacity());
+    }
+    int userLimit = conf.getUserLimit(queuePath);
+    float userLimitFactor = conf.getUserLimitFactor(queuePath);
+    int maxAppsPerUserForReservation =
+        (int) (maxAppsForReservation * (userLimit / 100.0f) * userLimitFactor);
+    updateQuotas(userLimit, userLimitFactor, maxAppsForReservation,
+        maxAppsPerUserForReservation);
+
+    StringBuffer queueInfo = new StringBuffer();
+    queueInfo.append("Created Plan Queue: ").append(queueName)
+        .append("\nwith capacity: [").append(super.getCapacity())
+        .append("]\nwith max capacity: [").append(super.getMaximumCapacity())
+        .append("\nwith max reservation apps: [").append(maxAppsForReservation)
+        .append("]\nwith max reservation apps per user: [")
+        .append(maxAppsPerUserForReservation).append("]\nwith user limit: [")
+        .append(userLimit).append("]\nwith user limit factor: [")
+        .append(userLimitFactor).append("].");
+    LOG.info(queueInfo.toString());
+  }
+
+  @Override
+  public synchronized void reinitialize(CSQueue newlyParsedQueue,
+      Resource clusterResource) throws IOException {
+    // Sanity check
+    if (!(newlyParsedQueue instanceof PlanQueue)
+        || !newlyParsedQueue.getQueuePath().equals(getQueuePath())) {
+      throw new IOException("Trying to reinitialize " + getQueuePath()
+          + " from " + newlyParsedQueue.getQueuePath());
+    }
+
+    PlanQueue newlyParsedParentQueue = (PlanQueue) newlyParsedQueue;
+
+    if (newlyParsedParentQueue.getChildQueues().size() > 0) {
+      throw new IOException(
+          "Reservable Queue should not have sub-queues in the"
+              + "configuration");
+    }
+
+    // Set new configs
+    setupQueueConfigs(clusterResource, newlyParsedParentQueue.getCapacity(),
+        newlyParsedParentQueue.getAbsoluteCapacity(),
+        newlyParsedParentQueue.getMaximumCapacity(),
+        newlyParsedParentQueue.getAbsoluteMaximumCapacity(),
+        newlyParsedParentQueue.getState(), newlyParsedParentQueue.getACLs());
+
+    updateQuotas(newlyParsedParentQueue.userLimit,
+        newlyParsedParentQueue.userLimitFactor,
+        newlyParsedParentQueue.maxAppsForReservation,
+        newlyParsedParentQueue.maxAppsPerUserForReservation);
+
+    // run reinitialize on each existing queue, to trigger absolute cap
+    // recomputations
+    for (CSQueue res : this.getChildQueues()) {
+      res.reinitialize(res, clusterResource);
+    }
+    showReservationsAsQueues = newlyParsedParentQueue.showReservationsAsQueues;
+  }
+
+  synchronized void addChildQueue(CSQueue newQueue)
+      throws SchedulerDynamicEditException {
+    if (newQueue.getCapacity() > 0) {
+      throw new SchedulerDynamicEditException("Queue " + newQueue
+          + " being added has non zero capacity.");
+    }
+    boolean added = this.childQueues.add(newQueue);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("updateChildQueues (action: add queue): " + added + " "
+          + getChildQueuesToPrint());
+    }
+  }
+
+  synchronized void removeChildQueue(CSQueue remQueue)
+      throws SchedulerDynamicEditException {
+    if (remQueue.getCapacity() > 0) {
+      throw new SchedulerDynamicEditException("Queue " + remQueue
+          + " being removed has non zero capacity.");
+    }
+    Iterator<CSQueue> qiter = childQueues.iterator();
+    while (qiter.hasNext()) {
+      CSQueue cs = qiter.next();
+      if (cs.equals(remQueue)) {
+        qiter.remove();
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Removed child queue: {}", cs.getQueueName());
+        }
+      }
+    }
+  }
+
+  protected synchronized float sumOfChildCapacities() {
+    float ret = 0;
+    for (CSQueue l : childQueues) {
+      ret += l.getCapacity();
+    }
+    return ret;
+  }
+
+  private void updateQuotas(int userLimit, float userLimitFactor,
+      int maxAppsForReservation, int maxAppsPerUserForReservation) {
+    this.userLimit = userLimit;
+    this.userLimitFactor = userLimitFactor;
+    this.maxAppsForReservation = maxAppsForReservation;
+    this.maxAppsPerUserForReservation = maxAppsPerUserForReservation;
+  }
+
+  /**
+   * Number of maximum applications for each of the reservations in this Plan.
+   *
+   * @return maxAppsForreservation
+   */
+  public int getMaxApplicationsForReservations() {
+    return maxAppsForReservation;
+  }
+
+  /**
+   * Number of maximum applications per user for each of the reservations in
+   * this Plan.
+   *
+   * @return maxAppsPerUserForreservation
+   */
+  public int getMaxApplicationsPerUserForReservation() {
+    return maxAppsPerUserForReservation;
+  }
+
+  /**
+   * User limit value for each of the reservations in this Plan.
+   *
+   * @return userLimit
+   */
+  public int getUserLimitForReservation() {
+    return userLimit;
+  }
+
+  /**
+   * User limit factor value for each of the reservations in this Plan.
+   *
+   * @return userLimitFactor
+   */
+  public float getUserLimitFactor() {
+    return userLimitFactor;
+  }
+
+  /**
+   * Determine whether to hide/show the ReservationQueues
+   */
+  public boolean showReservationsAsQueues() {
+    return showReservationsAsQueues;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb3e40b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java
new file mode 100644
index 0000000..48733fc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java
@@ -0,0 +1,98 @@
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import java.io.IOException;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This represents a dynamic {@link LeafQueue} managed by the
+ * {@link ReservationSystem}
+ *
+ */
+public class ReservationQueue extends LeafQueue {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ReservationQueue.class);
+
+  private PlanQueue parent;
+
+  private int maxSystemApps;
+
+  public ReservationQueue(CapacitySchedulerContext cs, String queueName,
+      PlanQueue parent) {
+    super(cs, queueName, parent, null);
+    maxSystemApps = cs.getConfiguration().getMaximumSystemApplications();
+    // the following parameters are common to all reservation in the plan
+    updateQuotas(parent.getUserLimitForReservation(),
+        parent.getUserLimitFactor(),
+        parent.getMaxApplicationsForReservations(),
+        parent.getMaxApplicationsPerUserForReservation());
+    this.parent = parent;
+  }
+
+  @Override
+  public synchronized void reinitialize(CSQueue newlyParsedQueue,
+      Resource clusterResource) throws IOException {
+    // Sanity check
+    if (!(newlyParsedQueue instanceof ReservationQueue)
+        || !newlyParsedQueue.getQueuePath().equals(getQueuePath())) {
+      throw new IOException("Trying to reinitialize " + getQueuePath()
+          + " from " + newlyParsedQueue.getQueuePath());
+    }
+    CSQueueUtils.updateQueueStatistics(
+        parent.schedulerContext.getResourceCalculator(), newlyParsedQueue,
+        parent, parent.schedulerContext.getClusterResource(),
+        parent.schedulerContext.getMinimumResourceCapability());
+    super.reinitialize(newlyParsedQueue, clusterResource);
+    updateQuotas(parent.getUserLimitForReservation(),
+        parent.getUserLimitFactor(),
+        parent.getMaxApplicationsForReservations(),
+        parent.getMaxApplicationsPerUserForReservation());
+  }
+
+  /**
+   * This methods to change capacity for a queue and adjusts its
+   * absoluteCapacity
+   * 
+   * @param entitlement the new entitlement for the queue (capacity,
+   *          maxCapacity, etc..)
+   * @throws SchedulerDynamicEditException
+   */
+  public synchronized void setEntitlement(QueueEntitlement entitlement)
+      throws SchedulerDynamicEditException {
+    float capacity = entitlement.getCapacity();
+    if (capacity < 0 || capacity > 1.0f) {
+      throw new SchedulerDynamicEditException(
+          "Capacity demand is not in the [0,1] range: " + capacity);
+    }
+    setCapacity(capacity);
+    setAbsoluteCapacity(getParent().getAbsoluteCapacity() * getCapacity());
+    setMaxApplications((int) (maxSystemApps * getAbsoluteCapacity()));
+    // note: we currently set maxCapacity to capacity
+    // this might be revised later
+    setMaxCapacity(entitlement.getMaxCapacity());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("successfully changed to " + capacity + " for queue "
+          + this.getQueueName());
+    }
+  }
+
+  private void updateQuotas(int userLimit, float userLimitFactor,
+      int maxAppsForReservation, int maxAppsPerUserForReservation) {
+    setUserLimit(userLimit);
+    setUserLimitFactor(userLimitFactor);
+    setMaxApplications(maxAppsForReservation);
+    maxApplicationsPerUser = maxAppsPerUserForReservation;
+  }
+
+  // used by the super constructor, we initialize to zero
+  protected float getCapacityFromConf() {
+    return 0f;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb3e40b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/QueueEntitlement.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/scheduler/common/QueueEntitlement.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/QueueEntitlement.java
new file mode 100644
index 0000000..a348e13
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/QueueEntitlement.java
@@ -0,0 +1,28 @@
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common;
+
+public class QueueEntitlement {
+
+  private float capacity;
+  private float maxCapacity;
+
+  public QueueEntitlement(float capacity, float maxCapacity){
+    this.setCapacity(capacity);
+    this.maxCapacity = maxCapacity;
+   }
+
+  public float getMaxCapacity() {
+    return maxCapacity;
+  }
+
+  public void setMaxCapacity(float maxCapacity) {
+    this.maxCapacity = maxCapacity;
+  }
+
+  public float getCapacity() {
+    return capacity;
+  }
+
+  public void setCapacity(float capacity) {
+    this.capacity = capacity;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb3e40b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.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/webapp/dao/CapacitySchedulerQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
index ac16ce0..0a8faad 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
@@ -25,6 +25,7 @@ import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PlanQueue;
 
 @XmlRootElement
 @XmlAccessorType(XmlAccessType.FIELD)
@@ -48,6 +49,7 @@ public class CapacitySchedulerQueueInfo {
   protected QueueState state;
   protected CapacitySchedulerQueueInfoList queues;
   protected ResourceInfo resourcesUsed;
+  private boolean hideReservationQueues = true;
 
   CapacitySchedulerQueueInfo() {
   };
@@ -69,6 +71,10 @@ public class CapacitySchedulerQueueInfo {
     queueName = q.getQueueName();
     state = q.getState();
     resourcesUsed = new ResourceInfo(q.getUsedResources());
+    if(q instanceof PlanQueue &&
+       ((PlanQueue)q).showReservationsAsQueues()) {
+      hideReservationQueues = false;
+    }
   }
 
   public float getCapacity() {
@@ -112,6 +118,9 @@ public class CapacitySchedulerQueueInfo {
   }
 
   public CapacitySchedulerQueueInfoList getQueues() {
+    if(hideReservationQueues) {
+      return new CapacitySchedulerQueueInfoList();
+    }
     return this.queues;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb3e40b8/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
----------------------------------------------------------------------
diff --git a/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 b/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
index e5f4df2..f7c098c 100644
--- a/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
+++ b/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
@@ -410,7 +410,7 @@ public class TestCapacityScheduler {
     cs.stop();
   }
 
-  private void checkQueueCapacities(CapacityScheduler cs,
+  void checkQueueCapacities(CapacityScheduler cs,
       float capacityA, float capacityB) {
     CSQueue rootQueue = cs.getRootQueue();
     CSQueue queueA = findQueue(rootQueue, A);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb3e40b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.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/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.java
new file mode 100644
index 0000000..aecbfa8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.java
@@ -0,0 +1,282 @@
+/**
+ * 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.capacity;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCapacitySchedulerDynamicBehavior {
+  private static final Log LOG = LogFactory
+      .getLog(TestCapacitySchedulerDynamicBehavior.class);
+  private static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+  private static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+  private static final String B1 = B + ".b1";
+  private static final String B2 = B + ".b2";
+  private static final String B3 = B + ".b3";
+  private static float A_CAPACITY = 10.5f;
+  private static float B_CAPACITY = 89.5f;
+  private static float A1_CAPACITY = 30;
+  private static float A2_CAPACITY = 70;
+  private static float B1_CAPACITY = 79.2f;
+  private static float B2_CAPACITY = 0.8f;
+  private static float B3_CAPACITY = 20;
+
+  private final TestCapacityScheduler tcs = new TestCapacityScheduler();
+
+  private int GB = 1024;
+
+  private MockRM rm;
+
+  @Before
+  public void setUp() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    setupPlanQueueConfiguration(conf);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.setBoolean(YarnConfiguration.RM_RESERVATIONS_ENABLE, false);
+    rm = new MockRM(conf);
+    rm.start();
+  }
+
+  @Test
+  public void testRefreshQueuesWithReservations() throws Exception {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    // Test add one reservation dynamically and manually modify capacity
+    ReservationQueue a1 =
+        new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
+    cs.addQueue(a1);
+    a1.setEntitlement(new QueueEntitlement(A1_CAPACITY / 100, 1f));
+
+    // Test add another reservation queue and use setEntitlement to modify
+    // capacity
+    ReservationQueue a2 =
+        new ReservationQueue(cs, "a2", (PlanQueue) cs.getQueue("a"));
+    cs.addQueue(a2);
+    cs.setEntitlement("a2", new QueueEntitlement(A2_CAPACITY / 100, 1.0f));
+
+    // Verify all allocations match
+    tcs.checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
+
+    // Reinitialize and verify all dynamic queued survived
+    CapacitySchedulerConfiguration conf = cs.getConfiguration();
+    conf.setCapacity(A, 80f);
+    conf.setCapacity(B, 20f);
+    cs.reinitialize(conf, rm.getRMContext());
+
+    tcs.checkQueueCapacities(cs, 80f, 20f);
+  }
+
+  @Test
+  public void testAddQueueFailCases() throws Exception {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    try {
+      // Test invalid addition (adding non-zero size queue)
+      ReservationQueue a1 =
+          new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
+      a1.setEntitlement(new QueueEntitlement(A1_CAPACITY / 100, 1f));
+      cs.addQueue(a1);
+      fail();
+    } catch (Exception e) {
+      // expected
+    }
+
+    // Test add one reservation dynamically and manually modify capacity
+    ReservationQueue a1 =
+        new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
+    cs.addQueue(a1);
+    a1.setEntitlement(new QueueEntitlement(A1_CAPACITY / 100, 1f));
+
+    // Test add another reservation queue and use setEntitlement to modify
+    // capacity
+    ReservationQueue a2 =
+        new ReservationQueue(cs, "a2", (PlanQueue) cs.getQueue("a"));
+
+    cs.addQueue(a2);
+
+    try {
+      // Test invalid entitlement (sum of queues exceed 100%)
+      cs.setEntitlement("a2", new QueueEntitlement(A2_CAPACITY / 100 + 0.1f,
+          1.0f));
+      fail();
+    } catch (Exception e) {
+      // expected
+    }
+
+    cs.setEntitlement("a2", new QueueEntitlement(A2_CAPACITY / 100, 1.0f));
+
+    // Verify all allocations match
+    tcs.checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
+
+    cs.stop();
+  }
+
+  @Test
+  public void testRemoveQueue() throws Exception {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    // Test add one reservation dynamically and manually modify capacity
+    ReservationQueue a1 =
+        new ReservationQueue(cs, "a1", (PlanQueue) cs.getQueue("a"));
+    cs.addQueue(a1);
+    a1.setEntitlement(new QueueEntitlement(A1_CAPACITY / 100, 1f));
+
+    // submit an app
+    RMApp app = rm.submitApp(GB, "test-move-1", "user_0", null, "a1");
+    // check preconditions
+    List<ApplicationAttemptId> appsInA1 = cs.getAppsInQueue("a1");
+    assertEquals(1, appsInA1.size());
+    try {
+      cs.removeQueue("a1");
+      fail();
+    } catch (SchedulerDynamicEditException s) {
+      // expected a1 contains applications
+    }
+    // clear queue by killling all apps
+    cs.killAllAppsInQueue("a1");
+    // wait for events of move to propagate
+    rm.waitForState(app.getApplicationId(), RMAppState.KILLED);
+
+    try {
+      cs.removeQueue("a1");
+      fail();
+    } catch (SchedulerDynamicEditException s) {
+      // expected a1 is not zero capacity
+    }
+    // set capacity to zero
+    cs.setEntitlement("a1", new QueueEntitlement(0f, 0f));
+    cs.removeQueue("a1");
+
+    assertTrue(cs.getQueue("a1") == null);
+
+    rm.stop();
+  }
+
+  @Test
+  public void testMoveAppToPlanQueue() throws Exception {
+    CapacityScheduler scheduler = (CapacityScheduler) rm.getResourceScheduler();
+
+    // submit an app
+    RMApp app = rm.submitApp(GB, "test-move-1", "user_0", null, "b1");
+    ApplicationAttemptId appAttemptId =
+        rm.getApplicationReport(app.getApplicationId())
+            .getCurrentApplicationAttemptId();
+
+    // check preconditions
+    List<ApplicationAttemptId> appsInB1 = scheduler.getAppsInQueue("b1");
+    assertEquals(1, appsInB1.size());
+
+    List<ApplicationAttemptId> appsInB = scheduler.getAppsInQueue("b");
+    assertEquals(1, appsInB.size());
+    assertTrue(appsInB.contains(appAttemptId));
+
+    List<ApplicationAttemptId> appsInA = scheduler.getAppsInQueue("a");
+    assertTrue(appsInA.isEmpty());
+
+    String queue =
+        scheduler.getApplicationAttempt(appsInB1.get(0)).getQueue()
+            .getQueueName();
+    Assert.assertTrue(queue.equals("b1"));
+
+    List<ApplicationAttemptId> appsInRoot = scheduler.getAppsInQueue("root");
+    assertTrue(appsInRoot.contains(appAttemptId));
+    assertEquals(1, appsInRoot.size());
+
+    // create the default reservation queue
+    String defQName = "a" + PlanQueue.DEFAULT_QUEUE_SUFFIX;
+    ReservationQueue defQ =
+        new ReservationQueue(scheduler, defQName,
+            (PlanQueue) scheduler.getQueue("a"));
+    scheduler.addQueue(defQ);
+    defQ.setEntitlement(new QueueEntitlement(1f, 1f));
+
+    List<ApplicationAttemptId> appsInDefQ = scheduler.getAppsInQueue(defQName);
+    assertTrue(appsInDefQ.isEmpty());
+
+    // now move the app to plan queue
+    scheduler.moveApplication(app.getApplicationId(), "a");
+
+    // check postconditions
+    appsInDefQ = scheduler.getAppsInQueue(defQName);
+    assertEquals(1, appsInDefQ.size());
+    queue =
+        scheduler.getApplicationAttempt(appsInDefQ.get(0)).getQueue()
+            .getQueueName();
+    Assert.assertTrue(queue.equals(defQName));
+
+    appsInA = scheduler.getAppsInQueue("a");
+    assertTrue(appsInA.contains(appAttemptId));
+    assertEquals(1, appsInA.size());
+
+    appsInRoot = scheduler.getAppsInQueue("root");
+    assertTrue(appsInRoot.contains(appAttemptId));
+    assertEquals(1, appsInRoot.size());
+
+    appsInB1 = scheduler.getAppsInQueue("b1");
+    assertTrue(appsInB1.isEmpty());
+
+    appsInB = scheduler.getAppsInQueue("b");
+    assertTrue(appsInB.isEmpty());
+
+    rm.stop();
+  }
+
+  private void setupPlanQueueConfiguration(CapacitySchedulerConfiguration conf) {
+
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] { "a", "b" });
+
+    conf.setCapacity(A, A_CAPACITY);
+    conf.setCapacity(B, B_CAPACITY);
+
+    // Define 2nd-level queues
+    conf.setQueues(B, new String[] { "b1", "b2", "b3" });
+    conf.setCapacity(B1, B1_CAPACITY);
+    conf.setUserLimitFactor(B1, 100.0f);
+    conf.setCapacity(B2, B2_CAPACITY);
+    conf.setUserLimitFactor(B2, 100.0f);
+    conf.setCapacity(B3, B3_CAPACITY);
+    conf.setUserLimitFactor(B3, 100.0f);
+
+    conf.setReservableQueue(A, true);
+    conf.setReservationWindow(A, 86400 * 1000);
+    conf.setAverageCapacity(A, 1.0f);
+
+    LOG.info("Setup a as a plan queue");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb3e40b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservationQueue.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/scheduler/capacity/TestReservationQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservationQueue.java
new file mode 100644
index 0000000..c53b7a9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservationQueue.java
@@ -0,0 +1,103 @@
+/**
+ * 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.capacity;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestReservationQueue {
+
+  CapacitySchedulerConfiguration csConf;
+  CapacitySchedulerContext csContext;
+  final static int GB = 1024;
+  private final ResourceCalculator resourceCalculator =
+      new DefaultResourceCalculator();
+  ReservationQueue reservationQueue;
+
+  @Before
+  public void setup() {
+
+    // setup a context / conf
+    csConf = new CapacitySchedulerConfiguration();
+    YarnConfiguration conf = new YarnConfiguration();
+    csContext = mock(CapacitySchedulerContext.class);
+    when(csContext.getConfiguration()).thenReturn(csConf);
+    when(csContext.getConf()).thenReturn(conf);
+    when(csContext.getMinimumResourceCapability()).thenReturn(
+        Resources.createResource(GB, 1));
+    when(csContext.getMaximumResourceCapability()).thenReturn(
+        Resources.createResource(16 * GB, 32));
+    when(csContext.getClusterResource()).thenReturn(
+        Resources.createResource(100 * 16 * GB, 100 * 32));
+    when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
+
+    // create a queue
+    PlanQueue pq = new PlanQueue(csContext, "root", null, null);
+    reservationQueue = new ReservationQueue(csContext, "a", pq);
+
+  }
+
+  @Test
+  public void testAddSubtractCapacity() throws Exception {
+
+    // verify that setting, adding, subtracting capacity works
+    reservationQueue.setCapacity(1.0F);
+    assertTrue(" actual capacity: " + reservationQueue.getCapacity(),
+        reservationQueue.getCapacity() - 1 < CSQueueUtils.EPSILON);
+    reservationQueue.setEntitlement(new QueueEntitlement(0.9f, 1f));
+    assertTrue(" actual capacity: " + reservationQueue.getCapacity(),
+        reservationQueue.getCapacity() - 0.9 < CSQueueUtils.EPSILON);
+    reservationQueue.setEntitlement(new QueueEntitlement(1f, 1f));
+    assertTrue(" actual capacity: " + reservationQueue.getCapacity(),
+        reservationQueue.getCapacity() - 1 < CSQueueUtils.EPSILON);
+    reservationQueue.setEntitlement(new QueueEntitlement(0f, 1f));
+    assertTrue(" actual capacity: " + reservationQueue.getCapacity(),
+        reservationQueue.getCapacity() < CSQueueUtils.EPSILON);
+
+    try {
+      reservationQueue.setEntitlement(new QueueEntitlement(1.1f, 1f));
+      fail();
+    } catch (SchedulerDynamicEditException iae) {
+      // expected
+      assertTrue(" actual capacity: " + reservationQueue.getCapacity(),
+          reservationQueue.getCapacity() - 1 < CSQueueUtils.EPSILON);
+    }
+
+    try {
+      reservationQueue.setEntitlement(new QueueEntitlement(-0.1f, 1f));
+      fail();
+    } catch (SchedulerDynamicEditException iae) {
+      // expected
+      assertTrue(" actual capacity: " + reservationQueue.getCapacity(),
+          reservationQueue.getCapacity() - 1 < CSQueueUtils.EPSILON);
+    }
+
+  }
+}


[10/16] YARN-2080. Integrating reservation system with ResourceManager and client-RM protocol. Contributed by Subru Krishnan and Carlo Curino. (cherry picked from commit 8baeaead8532898163f1006276b731a237b1a559)

Posted by cd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.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/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index db867a9..954e21d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -33,6 +33,7 @@ import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -74,6 +75,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -91,7 +98,12 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -107,6 +119,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWri
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystemTestUtil;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
@@ -117,11 +130,16 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.hadoop.yarn.util.UTCClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -1199,4 +1217,102 @@ public class TestClientRMService {
     when(yarnScheduler.getAppResourceUsageReport(attemptId)).thenReturn(null);
     return yarnScheduler;
   }
+
+  @Test
+  public void testReservationAPIs() {
+    // initialize
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    ReservationSystemTestUtil.setupQueueConfiguration(conf);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE, true);
+    MockRM rm = new MockRM(conf);
+    rm.start();
+    MockNM nm;
+    try {
+      nm = rm.registerNode("127.0.0.1:0", 102400, 100);
+      // allow plan follower to synchronize
+      Thread.sleep(1050);
+    } catch (Exception e) {
+      Assert.fail(e.getMessage());
+    }
+
+    // Create a client.
+    ClientRMService clientService = rm.getClientRMService();
+
+    // create a reservation
+    Clock clock = new UTCClock();
+    long arrival = clock.getTime();
+    long duration = 60000;
+    long deadline = (long) (arrival + 1.05 * duration);
+    ReservationSubmissionRequest sRequest =
+        createSimpleReservationRequest(4, arrival, deadline, duration);
+    ReservationSubmissionResponse sResponse = null;
+    try {
+      sResponse = clientService.submitReservation(sRequest);
+    } catch (Exception e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNotNull(sResponse);
+    ReservationId reservationID = sResponse.getReservationId();
+    Assert.assertNotNull(reservationID);
+    LOG.info("Submit reservation response: " + reservationID);
+
+    // Update the reservation
+    ReservationDefinition rDef = sRequest.getReservationDefinition();
+    ReservationRequest rr =
+        rDef.getReservationRequests().getReservationResources().get(0);
+    rr.setNumContainers(5);
+    arrival = clock.getTime();
+    duration = 30000;
+    deadline = (long) (arrival + 1.05 * duration);
+    rr.setDuration(duration);
+    rDef.setArrival(arrival);
+    rDef.setDeadline(deadline);
+    ReservationUpdateRequest uRequest =
+        ReservationUpdateRequest.newInstance(rDef, reservationID);
+    ReservationUpdateResponse uResponse = null;
+    try {
+      uResponse = clientService.updateReservation(uRequest);
+    } catch (Exception e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNotNull(sResponse);
+    LOG.info("Update reservation response: " + uResponse);
+
+    // Delete the reservation
+    ReservationDeleteRequest dRequest =
+        ReservationDeleteRequest.newInstance(reservationID);
+    ReservationDeleteResponse dResponse = null;
+    try {
+      dResponse = clientService.deleteReservation(dRequest);
+    } catch (Exception e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNotNull(sResponse);
+    LOG.info("Delete reservation response: " + dResponse);
+
+    // clean-up
+    rm.stop();
+    nm = null;
+    rm = null;
+  }
+
+  private ReservationSubmissionRequest createSimpleReservationRequest(
+      int numContainers, long arrival, long deadline, long duration) {
+    // create a request with a single atomic ask
+    ReservationRequest r =
+        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+            numContainers, 1, duration);
+    ReservationRequests reqs =
+        ReservationRequests.newInstance(Collections.singletonList(r),
+            ReservationRequestInterpreter.R_ALL);
+    ReservationDefinition rDef =
+        ReservationDefinition.newInstance(arrival, deadline, reqs,
+            "testClientRMService#reservation");
+    ReservationSubmissionRequest request =
+        ReservationSubmissionRequest.newInstance(rDef,
+            ReservationSystemTestUtil.reservationQ);
+    return request;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityReservationSystem.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/TestCapacityReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityReservationSystem.java
new file mode 100644
index 0000000..2a77791
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityReservationSystem.java
@@ -0,0 +1,102 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import java.io.IOException;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestCapacityReservationSystem {
+
+  @Test
+  public void testInitialize() {
+    ReservationSystemTestUtil testUtil = new ReservationSystemTestUtil();
+    CapacityScheduler capScheduler = null;
+    try {
+      capScheduler = testUtil.mockCapacityScheduler(10);
+    } catch (IOException e) {
+      Assert.fail(e.getMessage());
+    }
+    CapacityReservationSystem reservationSystem =
+        new CapacityReservationSystem();
+    reservationSystem.setRMContext(capScheduler.getRMContext());
+    try {
+      reservationSystem.reinitialize(capScheduler.getConf(),
+          capScheduler.getRMContext());
+    } catch (YarnException e) {
+      Assert.fail(e.getMessage());
+    }
+    String planQName = testUtil.getreservationQueueName();
+    Plan plan = reservationSystem.getPlan(planQName);
+    Assert.assertNotNull(plan);
+    Assert.assertTrue(plan instanceof InMemoryPlan);
+    Assert.assertEquals(planQName, plan.getQueueName());
+    Assert.assertEquals(8192, plan.getTotalCapacity().getMemory());
+    Assert
+        .assertTrue(plan.getReservationAgent() instanceof GreedyReservationAgent);
+    Assert
+        .assertTrue(plan.getSharingPolicy() instanceof CapacityOverTimePolicy);
+  }
+
+  @Test
+  public void testReinitialize() {
+    ReservationSystemTestUtil testUtil = new ReservationSystemTestUtil();
+    CapacityScheduler capScheduler = null;
+    try {
+      capScheduler = testUtil.mockCapacityScheduler(10);
+    } catch (IOException e) {
+      Assert.fail(e.getMessage());
+    }
+    CapacityReservationSystem reservationSystem =
+        new CapacityReservationSystem();
+    CapacitySchedulerConfiguration conf = capScheduler.getConfiguration();
+    RMContext mockContext = capScheduler.getRMContext();
+    reservationSystem.setRMContext(mockContext);
+    try {
+      reservationSystem.reinitialize(capScheduler.getConfiguration(),
+          mockContext);
+    } catch (YarnException e) {
+      Assert.fail(e.getMessage());
+    }
+    // Assert queue in original config
+    String planQName = testUtil.getreservationQueueName();
+    Plan plan = reservationSystem.getPlan(planQName);
+    Assert.assertNotNull(plan);
+    Assert.assertTrue(plan instanceof InMemoryPlan);
+    Assert.assertEquals(planQName, plan.getQueueName());
+    Assert.assertEquals(8192, plan.getTotalCapacity().getMemory());
+    Assert
+        .assertTrue(plan.getReservationAgent() instanceof GreedyReservationAgent);
+    Assert
+        .assertTrue(plan.getSharingPolicy() instanceof CapacityOverTimePolicy);
+
+    // Dynamically add a plan
+    String newQ = "reservation";
+    Assert.assertNull(reservationSystem.getPlan(newQ));
+    testUtil.updateQueueConfiguration(conf, newQ);
+    try {
+      capScheduler.reinitialize(conf, mockContext);
+    } catch (IOException e) {
+      Assert.fail(e.getMessage());
+    }
+    try {
+      reservationSystem.reinitialize(conf, mockContext);
+    } catch (YarnException e) {
+      Assert.fail(e.getMessage());
+    }
+    Plan newPlan = reservationSystem.getPlan(newQ);
+    Assert.assertNotNull(newPlan);
+    Assert.assertTrue(newPlan instanceof InMemoryPlan);
+    Assert.assertEquals(newQ, newPlan.getQueueName());
+    Assert.assertEquals(1024, newPlan.getTotalCapacity().getMemory());
+    Assert
+        .assertTrue(newPlan.getReservationAgent() instanceof GreedyReservationAgent);
+    Assert
+        .assertTrue(newPlan.getSharingPolicy() instanceof CapacityOverTimePolicy);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.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/TestReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
new file mode 100644
index 0000000..f5917bb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
@@ -0,0 +1,560 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.text.MessageFormat;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationDeleteRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationSubmissionRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationUpdateRequestPBImpl;
+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.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationDefinitionPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationRequestsPBImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestReservationInputValidator {
+
+  private static final Log LOG = LogFactory
+      .getLog(TestReservationInputValidator.class);
+
+  private static final String PLAN_NAME = "test-reservation";
+
+  private Clock clock;
+  private Map<String, Plan> plans = new HashMap<String, Plan>(1);
+  private ReservationSystem rSystem;
+  private Plan plan;
+
+  private ReservationInputValidator rrValidator;
+
+  @Before
+  public void setUp() {
+    clock = mock(Clock.class);
+    plan = mock(Plan.class);
+    rSystem = mock(ReservationSystem.class);
+    plans.put(PLAN_NAME, plan);
+    rrValidator = new ReservationInputValidator(clock);
+    when(clock.getTime()).thenReturn(1L);
+    ResourceCalculator rCalc = new DefaultResourceCalculator();
+    Resource resource = Resource.newInstance(10240, 10);
+    when(plan.getResourceCalculator()).thenReturn(rCalc);
+    when(plan.getTotalCapacity()).thenReturn(resource);
+    when(rSystem.getQueueForReservation(any(ReservationId.class))).thenReturn(
+        PLAN_NAME);
+    when(rSystem.getPlan(PLAN_NAME)).thenReturn(plan);
+  }
+
+  @After
+  public void tearDown() {
+    rrValidator = null;
+    clock = null;
+    plan = null;
+  }
+
+  @Test
+  public void testSubmitReservationNormal() {
+    ReservationSubmissionRequest request =
+        createSimpleReservationSubmissionRequest(1, 1, 1, 5, 3);
+    Plan plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationSubmissionRequest(rSystem, request,
+              ReservationSystemTestUtil.getNewReservationId());
+    } catch (YarnException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNotNull(plan);
+  }
+
+  @Test
+  public void testSubmitReservationDoesnotExist() {
+    ReservationSubmissionRequest request =
+        new ReservationSubmissionRequestPBImpl();
+    Plan plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationSubmissionRequest(rSystem, request,
+              ReservationSystemTestUtil.getNewReservationId());
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert
+          .assertTrue(message
+              .equals("The queue to submit is not specified. Please try again with a valid reservable queue."));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testSubmitReservationInvalidPlan() {
+    ReservationSubmissionRequest request =
+        createSimpleReservationSubmissionRequest(1, 1, 1, 5, 3);
+    when(rSystem.getPlan(PLAN_NAME)).thenReturn(null);
+    Plan plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationSubmissionRequest(rSystem, request,
+              ReservationSystemTestUtil.getNewReservationId());
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert
+          .assertTrue(message
+              .endsWith(" is not managed by reservation system. Please try again with a valid reservable queue."));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testSubmitReservationNoDefinition() {
+    ReservationSubmissionRequest request =
+        new ReservationSubmissionRequestPBImpl();
+    request.setQueue(PLAN_NAME);
+    Plan plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationSubmissionRequest(rSystem, request,
+              ReservationSystemTestUtil.getNewReservationId());
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert
+          .assertTrue(message
+              .equals("Missing reservation definition. Please try again by specifying a reservation definition."));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testSubmitReservationInvalidDeadline() {
+    ReservationSubmissionRequest request =
+        createSimpleReservationSubmissionRequest(1, 1, 1, 0, 3);
+    Plan plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationSubmissionRequest(rSystem, request,
+              ReservationSystemTestUtil.getNewReservationId());
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("The specified deadline: 0 is the past"));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testSubmitReservationInvalidRR() {
+    ReservationSubmissionRequest request =
+        createSimpleReservationSubmissionRequest(0, 0, 1, 5, 3);
+    Plan plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationSubmissionRequest(rSystem, request,
+              ReservationSystemTestUtil.getNewReservationId());
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("No resources have been specified to reserve"));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testSubmitReservationEmptyRR() {
+    ReservationSubmissionRequest request =
+        createSimpleReservationSubmissionRequest(1, 0, 1, 5, 3);
+    Plan plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationSubmissionRequest(rSystem, request,
+              ReservationSystemTestUtil.getNewReservationId());
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("No resources have been specified to reserve"));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testSubmitReservationInvalidDuration() {
+    ReservationSubmissionRequest request =
+        createSimpleReservationSubmissionRequest(1, 1, 1, 3, 4);
+    Plan plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationSubmissionRequest(rSystem, request,
+              ReservationSystemTestUtil.getNewReservationId());
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message.startsWith("The time difference"));
+      Assert
+          .assertTrue(message
+              .contains("must  be greater or equal to the minimum resource duration"));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testSubmitReservationExceedsGangSize() {
+    ReservationSubmissionRequest request =
+        createSimpleReservationSubmissionRequest(1, 1, 1, 5, 4);
+    Resource resource = Resource.newInstance(512, 1);
+    when(plan.getTotalCapacity()).thenReturn(resource);
+    Plan plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationSubmissionRequest(rSystem, request,
+              ReservationSystemTestUtil.getNewReservationId());
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert
+          .assertTrue(message
+              .startsWith("The size of the largest gang in the reservation refinition"));
+      Assert.assertTrue(message.contains("exceed the capacity available "));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testUpdateReservationNormal() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 5, 3);
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
+    } catch (YarnException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNotNull(plan);
+  }
+
+  @Test
+  public void testUpdateReservationNoID() {
+    ReservationUpdateRequest request = new ReservationUpdateRequestPBImpl();
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert
+          .assertTrue(message
+              .startsWith("Missing reservation id. Please try again by specifying a reservation id."));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testUpdateReservationDoesnotExist() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 5, 4);
+    ReservationId rId = request.getReservationId();
+    when(rSystem.getQueueForReservation(rId)).thenReturn(null);
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert
+          .assertTrue(message.equals(MessageFormat
+              .format(
+                  "The specified reservation with ID: {0} is unknown. Please try again with a valid reservation.",
+                  rId)));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testUpdateReservationInvalidPlan() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 5, 4);
+    when(rSystem.getPlan(PLAN_NAME)).thenReturn(null);
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert
+          .assertTrue(message
+              .endsWith(" is not associated with any valid plan. Please try again with a valid reservation."));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testUpdateReservationNoDefinition() {
+    ReservationUpdateRequest request = new ReservationUpdateRequestPBImpl();
+    request.setReservationId(ReservationSystemTestUtil.getNewReservationId());
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert
+          .assertTrue(message
+              .startsWith("Missing reservation definition. Please try again by specifying a reservation definition."));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testUpdateReservationInvalidDeadline() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 0, 3);
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("The specified deadline: 0 is the past"));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testUpdateReservationInvalidRR() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(0, 0, 1, 5, 3);
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("No resources have been specified to reserve"));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testUpdateReservationEmptyRR() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 0, 1, 5, 3);
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("No resources have been specified to reserve"));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testUpdateReservationInvalidDuration() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 3, 4);
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert
+          .assertTrue(message
+              .contains("must  be greater or equal to the minimum resource duration"));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testUpdateReservationExceedsGangSize() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 5, 4);
+    Resource resource = Resource.newInstance(512, 1);
+    when(plan.getTotalCapacity()).thenReturn(resource);
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert
+          .assertTrue(message
+              .startsWith("The size of the largest gang in the reservation refinition"));
+      Assert.assertTrue(message.contains("exceed the capacity available "));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testDeleteReservationNormal() {
+    ReservationDeleteRequest request = new ReservationDeleteRequestPBImpl();
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    request.setReservationId(reservationID);
+    ReservationAllocation reservation = mock(ReservationAllocation.class);
+    when(plan.getReservationById(reservationID)).thenReturn(reservation);
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationDeleteRequest(rSystem, request);
+    } catch (YarnException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNotNull(plan);
+  }
+
+  @Test
+  public void testDeleteReservationNoID() {
+    ReservationDeleteRequest request = new ReservationDeleteRequestPBImpl();
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationDeleteRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert
+          .assertTrue(message
+              .startsWith("Missing reservation id. Please try again by specifying a reservation id."));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testDeleteReservationDoesnotExist() {
+    ReservationDeleteRequest request = new ReservationDeleteRequestPBImpl();
+    ReservationId rId = ReservationSystemTestUtil.getNewReservationId();
+    request.setReservationId(rId);
+    when(rSystem.getQueueForReservation(rId)).thenReturn(null);
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationDeleteRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert
+          .assertTrue(message.equals(MessageFormat
+              .format(
+                  "The specified reservation with ID: {0} is unknown. Please try again with a valid reservation.",
+                  rId)));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testDeleteReservationInvalidPlan() {
+    ReservationDeleteRequest request = new ReservationDeleteRequestPBImpl();
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    request.setReservationId(reservationID);
+    when(rSystem.getPlan(PLAN_NAME)).thenReturn(null);
+    Plan plan = null;
+    try {
+      plan = rrValidator.validateReservationDeleteRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert
+          .assertTrue(message
+              .endsWith(" is not associated with any valid plan. Please try again with a valid reservation."));
+      LOG.info(message);
+    }
+  }
+
+  private ReservationSubmissionRequest createSimpleReservationSubmissionRequest(
+      int numRequests, int numContainers, long arrival, long deadline,
+      long duration) {
+    // create a request with a single atomic ask
+    ReservationSubmissionRequest request =
+        new ReservationSubmissionRequestPBImpl();
+    ReservationDefinition rDef = new ReservationDefinitionPBImpl();
+    rDef.setArrival(arrival);
+    rDef.setDeadline(deadline);
+    if (numRequests > 0) {
+      ReservationRequests reqs = new ReservationRequestsPBImpl();
+      rDef.setReservationRequests(reqs);
+      if (numContainers > 0) {
+        ReservationRequest r =
+            ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+                numContainers, 1, duration);
+
+        reqs.setReservationResources(Collections.singletonList(r));
+        reqs.setInterpreter(ReservationRequestInterpreter.R_ALL);
+      }
+    }
+    request.setQueue(PLAN_NAME);
+    request.setReservationDefinition(rDef);
+    return request;
+  }
+
+  private ReservationUpdateRequest createSimpleReservationUpdateRequest(
+      int numRequests, int numContainers, long arrival, long deadline,
+      long duration) {
+    // create a request with a single atomic ask
+    ReservationUpdateRequest request = new ReservationUpdateRequestPBImpl();
+    ReservationDefinition rDef = new ReservationDefinitionPBImpl();
+    rDef.setArrival(arrival);
+    rDef.setDeadline(deadline);
+    if (numRequests > 0) {
+      ReservationRequests reqs = new ReservationRequestsPBImpl();
+      rDef.setReservationRequests(reqs);
+      if (numContainers > 0) {
+        ReservationRequest r =
+            ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+                numContainers, 1, duration);
+
+        reqs.setReservationResources(Collections.singletonList(r));
+        reqs.setInterpreter(ReservationRequestInterpreter.R_ALL);
+      }
+    }
+    request.setReservationDefinition(rDef);
+    request.setReservationId(ReservationSystemTestUtil.getNewReservationId());
+    return request;
+  }
+
+}


[05/16] YARN-1709. In-memory data structures used to track resources over time to enable reservations. (cherry picked from commit 0d8b2cd88b958b1e602fd4ea4078ef8d4742a7c3)

Posted by cd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/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
new file mode 100644
index 0000000..6dcd41f
--- /dev/null
+++ 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
@@ -0,0 +1,477 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+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.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationDefinitionPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationRequestsPBImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestInMemoryPlan {
+
+  private String user = "yarn";
+  private String planName = "test-reservation";
+  private ResourceCalculator resCalc;
+  private Resource minAlloc;
+  private Resource maxAlloc;
+  private Resource totalCapacity;
+
+  private Clock clock;
+  private QueueMetrics queueMetrics;
+  private SharingPolicy policy;
+  private ReservationAgent agent;
+  private Planner replanner;
+
+  @Before
+  public void setUp() throws PlanningException {
+    resCalc = new DefaultResourceCalculator();
+    minAlloc = Resource.newInstance(1024, 1);
+    maxAlloc = Resource.newInstance(64 * 1024, 20);
+    totalCapacity = Resource.newInstance(100 * 1024, 100);
+
+    clock = mock(Clock.class);
+    queueMetrics = mock(QueueMetrics.class);
+    policy = mock(SharingPolicy.class);
+    replanner = mock(Planner.class);
+
+    when(clock.getTime()).thenReturn(1L);
+  }
+
+  @After
+  public void tearDown() {
+    resCalc = null;
+    minAlloc = null;
+    maxAlloc = null;
+    totalCapacity = null;
+
+    clock = null;
+    queueMetrics = null;
+    policy = null;
+    replanner = null;
+  }
+
+  @Test
+  public void testAddReservation() {
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true);
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+    int start = 100;
+    Map<ReservationInterval, ReservationRequest> allocations =
+        generateAllocation(start, alloc, false);
+    ReservationDefinition rDef =
+        createSimpleReservationDefinition(start, start + alloc.length,
+            alloc.length, allocations.values());
+    ReservationAllocation rAllocation =
+        new InMemoryReservationAllocation(reservationID, rDef, user, planName,
+            start, start + alloc.length, allocations, resCalc, minAlloc);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    doAssertions(plan, rAllocation);
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+          plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+          plan.getConsumptionForUser(user, start + i));
+    }
+  }
+
+  @Test
+  public void testAddEmptyReservation() {
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true);
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = {};
+    int start = 100;
+    Map<ReservationInterval, ReservationRequest> allocations =
+        new HashMap<ReservationInterval, ReservationRequest>();
+    ReservationDefinition rDef =
+        createSimpleReservationDefinition(start, start + alloc.length,
+            alloc.length, allocations.values());
+    ReservationAllocation rAllocation =
+        new InMemoryReservationAllocation(reservationID, rDef, user, planName,
+            start, start + alloc.length, allocations, resCalc, minAlloc);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testAddReservationAlreadyExists() {
+    // First add a reservation
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true);
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+    int start = 100;
+    Map<ReservationInterval, ReservationRequest> allocations =
+        generateAllocation(start, alloc, false);
+    ReservationDefinition rDef =
+        createSimpleReservationDefinition(start, start + alloc.length,
+            alloc.length, allocations.values());
+    ReservationAllocation rAllocation =
+        new InMemoryReservationAllocation(reservationID, rDef, user, planName,
+            start, start + alloc.length, allocations, resCalc, minAlloc);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    doAssertions(plan, rAllocation);
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+          plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+          plan.getConsumptionForUser(user, start + i));
+    }
+
+    // Try to add it again
+    try {
+      plan.addReservation(rAllocation);
+      Assert.fail("Add should fail as it already exists");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage().endsWith("already exists"));
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    doAssertions(plan, rAllocation);
+  }
+
+  @Test
+  public void testUpdateReservation() {
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true);
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    // First add a reservation
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+    int start = 100;
+    Map<ReservationInterval, ReservationRequest> allocations =
+        generateAllocation(start, alloc, false);
+    ReservationDefinition rDef =
+        createSimpleReservationDefinition(start, start + alloc.length,
+            alloc.length, allocations.values());
+    ReservationAllocation rAllocation =
+        new InMemoryReservationAllocation(reservationID, rDef, user, planName,
+            start, start + alloc.length, allocations, resCalc, minAlloc);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    doAssertions(plan, rAllocation);
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+          plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+          plan.getConsumptionForUser(user, start + i));
+    }
+
+    // Now update it
+    start = 110;
+    int[] updatedAlloc = { 0, 5, 10, 10, 5, 0 };
+    allocations = generateAllocation(start, updatedAlloc, true);
+    rDef =
+        createSimpleReservationDefinition(start, start + updatedAlloc.length,
+            updatedAlloc.length, allocations.values());
+    rAllocation =
+        new InMemoryReservationAllocation(reservationID, rDef, user, planName,
+            start, start + updatedAlloc.length, allocations, resCalc, minAlloc);
+    try {
+      plan.updateReservation(rAllocation);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    doAssertions(plan, rAllocation);
+    for (int i = 0; i < updatedAlloc.length; i++) {
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
+              + i), plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (updatedAlloc[i] + i), updatedAlloc[i]
+              + i), plan.getConsumptionForUser(user, start + i));
+    }
+  }
+
+  @Test
+  public void testUpdateNonExistingReservation() {
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true);
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    // Try to update a reservation without adding
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+    int start = 100;
+    Map<ReservationInterval, ReservationRequest> allocations =
+        generateAllocation(start, alloc, false);
+    ReservationDefinition rDef =
+        createSimpleReservationDefinition(start, start + alloc.length,
+            alloc.length, allocations.values());
+    ReservationAllocation rAllocation =
+        new InMemoryReservationAllocation(reservationID, rDef, user, planName,
+            start, start + alloc.length, allocations, resCalc, minAlloc);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.updateReservation(rAllocation);
+      Assert.fail("Update should fail as it does not exist in the plan");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage().endsWith("does not exist in the plan"));
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNull(plan.getReservationById(reservationID));
+  }
+
+  @Test
+  public void testDeleteReservation() {
+    // First add a reservation
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true);
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+    int start = 100;
+    Map<ReservationInterval, ReservationRequest> allocations =
+        generateAllocation(start, alloc, true);
+    ReservationDefinition rDef =
+        createSimpleReservationDefinition(start, start + alloc.length,
+            alloc.length, allocations.values());
+    ReservationAllocation rAllocation =
+        new InMemoryReservationAllocation(reservationID, rDef, user, planName,
+            start, start + alloc.length, allocations, resCalc, minAlloc);
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.addReservation(rAllocation);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    doAssertions(plan, rAllocation);
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (alloc[i] + i), (alloc[i] + i)),
+          plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (alloc[i] + i), (alloc[i] + i)),
+          plan.getConsumptionForUser(user, start + i));
+    }
+
+    // Now delete it
+    try {
+      plan.deleteReservation(reservationID);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNull(plan.getReservationById(reservationID));
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(Resource.newInstance(0, 0),
+          plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(Resource.newInstance(0, 0),
+          plan.getConsumptionForUser(user, start + i));
+    }
+  }
+
+  @Test
+  public void testDeleteNonExistingReservation() {
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true);
+    ReservationId reservationID =
+        ReservationSystemTestUtil.getNewReservationId();
+    // Try to delete a reservation without adding
+    Assert.assertNull(plan.getReservationById(reservationID));
+    try {
+      plan.deleteReservation(reservationID);
+      Assert.fail("Delete should fail as it does not exist in the plan");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage().endsWith("does not exist in the plan"));
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNull(plan.getReservationById(reservationID));
+  }
+
+  @Test
+  public void testArchiveCompletedReservations() {
+    Plan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+            resCalc, minAlloc, maxAlloc, planName, replanner, true);
+    ReservationId reservationID1 =
+        ReservationSystemTestUtil.getNewReservationId();
+    // First add a reservation
+    int[] alloc1 = { 10, 10, 10, 10, 10, 10 };
+    int start = 100;
+    Map<ReservationInterval, ReservationRequest> allocations1 =
+        generateAllocation(start, alloc1, false);
+    ReservationDefinition rDef1 =
+        createSimpleReservationDefinition(start, start + alloc1.length,
+            alloc1.length, allocations1.values());
+    ReservationAllocation rAllocation =
+        new InMemoryReservationAllocation(reservationID1, rDef1, user,
+            planName, start, start + alloc1.length, allocations1, resCalc,
+            minAlloc);
+    Assert.assertNull(plan.getReservationById(reservationID1));
+    try {
+      plan.addReservation(rAllocation);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    doAssertions(plan, rAllocation);
+    for (int i = 0; i < alloc1.length; i++) {
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (alloc1[i]), (alloc1[i])),
+          plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (alloc1[i]), (alloc1[i])),
+          plan.getConsumptionForUser(user, start + i));
+    }
+
+    // Now add another one
+    ReservationId reservationID2 =
+        ReservationSystemTestUtil.getNewReservationId();
+    int[] alloc2 = { 0, 5, 10, 5, 0 };
+    Map<ReservationInterval, ReservationRequest> allocations2 =
+        generateAllocation(start, alloc2, true);
+    ReservationDefinition rDef2 =
+        createSimpleReservationDefinition(start, start + alloc2.length,
+            alloc2.length, allocations2.values());
+    rAllocation =
+        new InMemoryReservationAllocation(reservationID2, rDef2, user,
+            planName, start, start + alloc2.length, allocations2, resCalc,
+            minAlloc);
+    Assert.assertNull(plan.getReservationById(reservationID2));
+    try {
+      plan.addReservation(rAllocation);
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNotNull(plan.getReservationById(reservationID2));
+    for (int i = 0; i < alloc2.length; i++) {
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
+              + alloc2[i] + i), plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (alloc1[i] + alloc2[i] + i), alloc1[i]
+              + alloc2[i] + i), plan.getConsumptionForUser(user, start + i));
+    }
+
+    // Now archive completed reservations
+    when(clock.getTime()).thenReturn(106L);
+    when(policy.getValidWindow()).thenReturn(1L);
+    try {
+      // will only remove 2nd reservation as only that has fallen out of the
+      // archival window
+      plan.archiveCompletedReservations(clock.getTime());
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNotNull(plan.getReservationById(reservationID1));
+    Assert.assertNull(plan.getReservationById(reservationID2));
+    for (int i = 0; i < alloc1.length; i++) {
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (alloc1[i]), (alloc1[i])),
+          plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (alloc1[i]), (alloc1[i])),
+          plan.getConsumptionForUser(user, start + i));
+    }
+    when(clock.getTime()).thenReturn(107L);
+    try {
+      // will remove 1st reservation also as it has fallen out of the archival
+      // window
+      plan.archiveCompletedReservations(clock.getTime());
+    } catch (PlanningException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNull(plan.getReservationById(reservationID1));
+    for (int i = 0; i < alloc1.length; i++) {
+      Assert.assertEquals(Resource.newInstance(0, 0),
+          plan.getTotalCommittedResources(start + i));
+      Assert.assertEquals(Resource.newInstance(0, 0),
+          plan.getConsumptionForUser(user, start + i));
+    }
+  }
+
+  private void doAssertions(Plan plan, ReservationAllocation rAllocation) {
+    ReservationId reservationID = rAllocation.getReservationId();
+    Assert.assertNotNull(plan.getReservationById(reservationID));
+    Assert.assertEquals(rAllocation, plan.getReservationById(reservationID));
+    Assert.assertTrue(((InMemoryPlan) plan).getAllReservations().size() == 1);
+    Assert.assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
+    Assert.assertEquals(totalCapacity, plan.getTotalCapacity());
+    Assert.assertEquals(minAlloc, plan.getMinimumAllocation());
+    Assert.assertEquals(maxAlloc, plan.getMaximumAllocation());
+    Assert.assertEquals(resCalc, plan.getResourceCalculator());
+    Assert.assertEquals(planName, plan.getQueueName());
+    Assert.assertTrue(plan.getMoveOnExpiry());
+  }
+
+  private ReservationDefinition createSimpleReservationDefinition(long arrival,
+      long deadline, long duration, Collection<ReservationRequest> resources) {
+    // create a request with a single atomic ask
+    ReservationDefinition rDef = new ReservationDefinitionPBImpl();
+    ReservationRequests reqs = new ReservationRequestsPBImpl();
+    reqs.setReservationResources(new ArrayList<ReservationRequest>(resources));
+    reqs.setInterpreter(ReservationRequestInterpreter.R_ALL);
+    rDef.setReservationRequests(reqs);
+    rDef.setArrival(arrival);
+    rDef.setDeadline(deadline);
+    return rDef;
+  }
+
+  private Map<ReservationInterval, ReservationRequest> generateAllocation(
+      int startTime, int[] alloc, boolean isStep) {
+    Map<ReservationInterval, ReservationRequest> req =
+        new HashMap<ReservationInterval, ReservationRequest>();
+    int numContainers = 0;
+    for (int i = 0; i < alloc.length; i++) {
+      if (isStep) {
+        numContainers = alloc[i] + i;
+      } else {
+        numContainers = alloc[i];
+      }
+      ReservationRequest rr =
+          ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+              (numContainers));
+      req.put(new ReservationInterval(startTime + i, startTime + i + 1), rr);
+    }
+    return req;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/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
new file mode 100644
index 0000000..f4c4581
--- /dev/null
+++ 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
@@ -0,0 +1,206 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+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.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationDefinitionPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationRequestsPBImpl;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestInMemoryReservationAllocation {
+
+  private String user = "yarn";
+  private String planName = "test-reservation";
+  private ResourceCalculator resCalc;
+  private Resource minAlloc;
+
+  private Random rand = new Random();
+
+  @Before
+  public void setUp() {
+    resCalc = new DefaultResourceCalculator();
+    minAlloc = Resource.newInstance(1, 1);
+  }
+
+  @After
+  public void tearDown() {
+    user = null;
+    planName = null;
+    resCalc = null;
+    minAlloc = null;
+  }
+
+  @Test
+  public void testBlocks() {
+    ReservationId reservationID =
+        ReservationId.newInstance(rand.nextLong(), rand.nextLong());
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+    int start = 100;
+    ReservationDefinition rDef =
+        createSimpleReservationDefinition(start, start + alloc.length + 1,
+            alloc.length);
+    Map<ReservationInterval, ReservationRequest> allocations =
+        generateAllocation(start, alloc, false, false);
+    ReservationAllocation rAllocation =
+        new InMemoryReservationAllocation(reservationID, rDef, user, planName,
+            start, start + alloc.length + 1, allocations, resCalc, minAlloc);
+    doAssertions(rAllocation, reservationID, rDef, allocations, start, alloc);
+    Assert.assertFalse(rAllocation.containsGangs());
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+          rAllocation.getResourcesAtTime(start + i));
+    }
+  }
+
+  @Test
+  public void testSteps() {
+    ReservationId reservationID =
+        ReservationId.newInstance(rand.nextLong(), rand.nextLong());
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+    int start = 100;
+    ReservationDefinition rDef =
+        createSimpleReservationDefinition(start, start + alloc.length + 1,
+            alloc.length);
+    Map<ReservationInterval, ReservationRequest> allocations =
+        generateAllocation(start, alloc, true, false);
+    ReservationAllocation rAllocation =
+        new InMemoryReservationAllocation(reservationID, rDef, user, planName,
+            start, start + alloc.length + 1, allocations, resCalc, minAlloc);
+    doAssertions(rAllocation, reservationID, rDef, allocations, start, alloc);
+    Assert.assertFalse(rAllocation.containsGangs());
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (alloc[i] + i), (alloc[i] + i)),
+          rAllocation.getResourcesAtTime(start + i));
+    }
+  }
+
+  @Test
+  public void testSkyline() {
+    ReservationId reservationID =
+        ReservationId.newInstance(rand.nextLong(), rand.nextLong());
+    int[] alloc = { 0, 5, 10, 10, 5, 0 };
+    int start = 100;
+    ReservationDefinition rDef =
+        createSimpleReservationDefinition(start, start + alloc.length + 1,
+            alloc.length);
+    Map<ReservationInterval, ReservationRequest> allocations =
+        generateAllocation(start, alloc, true, false);
+    ReservationAllocation rAllocation =
+        new InMemoryReservationAllocation(reservationID, rDef, user, planName,
+            start, start + alloc.length + 1, allocations, resCalc, minAlloc);
+    doAssertions(rAllocation, reservationID, rDef, allocations, start, alloc);
+    Assert.assertFalse(rAllocation.containsGangs());
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (alloc[i] + i), (alloc[i] + i)),
+          rAllocation.getResourcesAtTime(start + i));
+    }
+  }
+
+  @Test
+  public void testZeroAlloaction() {
+    ReservationId reservationID =
+        ReservationId.newInstance(rand.nextLong(), rand.nextLong());
+    int[] alloc = {};
+    long start = 0;
+    ReservationDefinition rDef =
+        createSimpleReservationDefinition(start, start + alloc.length + 1,
+            alloc.length);
+    Map<ReservationInterval, ReservationRequest> allocations =
+        new HashMap<ReservationInterval, ReservationRequest>();
+    ReservationAllocation rAllocation =
+        new InMemoryReservationAllocation(reservationID, rDef, user, planName,
+            start, start + alloc.length + 1, allocations, resCalc, minAlloc);
+    doAssertions(rAllocation, reservationID, rDef, allocations, (int) start,
+        alloc);
+    Assert.assertFalse(rAllocation.containsGangs());
+  }
+
+  @Test
+  public void testGangAlloaction() {
+    ReservationId reservationID =
+        ReservationId.newInstance(rand.nextLong(), rand.nextLong());
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+    int start = 100;
+    ReservationDefinition rDef =
+        createSimpleReservationDefinition(start, start + alloc.length + 1,
+            alloc.length);
+    Map<ReservationInterval, ReservationRequest> allocations =
+        generateAllocation(start, alloc, false, true);
+    ReservationAllocation rAllocation =
+        new InMemoryReservationAllocation(reservationID, rDef, user, planName,
+            start, start + alloc.length + 1, allocations, resCalc, minAlloc);
+    doAssertions(rAllocation, reservationID, rDef, allocations, start, alloc);
+    Assert.assertTrue(rAllocation.containsGangs());
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+          rAllocation.getResourcesAtTime(start + i));
+    }
+  }
+
+  private void doAssertions(ReservationAllocation rAllocation,
+      ReservationId reservationID, ReservationDefinition rDef,
+      Map<ReservationInterval, ReservationRequest> allocations, int start,
+      int[] alloc) {
+    Assert.assertEquals(reservationID, rAllocation.getReservationId());
+    Assert.assertEquals(rDef, rAllocation.getReservationDefinition());
+    Assert.assertEquals(allocations, rAllocation.getAllocationRequests());
+    Assert.assertEquals(user, rAllocation.getUser());
+    Assert.assertEquals(planName, rAllocation.getPlanName());
+    Assert.assertEquals(start, rAllocation.getStartTime());
+    Assert.assertEquals(start + alloc.length + 1, rAllocation.getEndTime());
+  }
+
+  private ReservationDefinition createSimpleReservationDefinition(long arrival,
+      long deadline, long duration) {
+    // create a request with a single atomic ask
+    ReservationRequest r =
+        ReservationRequest.newInstance(Resource.newInstance(1024, 1), 1, 1,
+            duration);
+    ReservationDefinition rDef = new ReservationDefinitionPBImpl();
+    ReservationRequests reqs = new ReservationRequestsPBImpl();
+    reqs.setReservationResources(Collections.singletonList(r));
+    reqs.setInterpreter(ReservationRequestInterpreter.R_ALL);
+    rDef.setReservationRequests(reqs);
+    rDef.setArrival(arrival);
+    rDef.setDeadline(deadline);
+    return rDef;
+  }
+
+  private Map<ReservationInterval, ReservationRequest> generateAllocation(
+      int startTime, int[] alloc, boolean isStep, boolean isGang) {
+    Map<ReservationInterval, ReservationRequest> req =
+        new HashMap<ReservationInterval, ReservationRequest>();
+    int numContainers = 0;
+    for (int i = 0; i < alloc.length; i++) {
+      if (isStep) {
+        numContainers = alloc[i] + i;
+      } else {
+        numContainers = alloc[i];
+      }
+      ReservationRequest rr =
+          ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+              (numContainers));
+      if (isGang) {
+        rr.setConcurrency(numContainers);
+      }
+      req.put(new ReservationInterval(startTime + i, startTime + i + 1), rr);
+    }
+    return req;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/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
new file mode 100644
index 0000000..ab0de6b
--- /dev/null
+++ 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
@@ -0,0 +1,169 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestRLESparseResourceAllocation {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestRLESparseResourceAllocation.class);
+
+  @Test
+  public void testBlocks() {
+    ResourceCalculator resCalc = new DefaultResourceCalculator();
+    Resource minAlloc = Resource.newInstance(1, 1);
+
+    RLESparseResourceAllocation rleSparseVector =
+        new RLESparseResourceAllocation(resCalc, minAlloc);
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+    int start = 100;
+    Set<Entry<ReservationInterval, ReservationRequest>> inputs =
+        generateAllocation(start, alloc, false).entrySet();
+    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+      rleSparseVector.addInterval(ip.getKey(), ip.getValue());
+    }
+    LOG.info(rleSparseVector.toString());
+    Assert.assertFalse(rleSparseVector.isEmpty());
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        rleSparseVector.getCapacityAtTime(99));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        rleSparseVector.getCapacityAtTime(start + alloc.length + 1));
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+          rleSparseVector.getCapacityAtTime(start + i));
+    }
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
+    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+      rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
+    }
+    LOG.info(rleSparseVector.toString());
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(Resource.newInstance(0, 0),
+          rleSparseVector.getCapacityAtTime(start + i));
+    }
+    Assert.assertTrue(rleSparseVector.isEmpty());
+  }
+
+  @Test
+  public void testSteps() {
+    ResourceCalculator resCalc = new DefaultResourceCalculator();
+    Resource minAlloc = Resource.newInstance(1, 1);
+
+    RLESparseResourceAllocation rleSparseVector =
+        new RLESparseResourceAllocation(resCalc, minAlloc);
+    int[] alloc = { 10, 10, 10, 10, 10, 10 };
+    int start = 100;
+    Set<Entry<ReservationInterval, ReservationRequest>> inputs =
+        generateAllocation(start, alloc, true).entrySet();
+    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+      rleSparseVector.addInterval(ip.getKey(), ip.getValue());
+    }
+    LOG.info(rleSparseVector.toString());
+    Assert.assertFalse(rleSparseVector.isEmpty());
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        rleSparseVector.getCapacityAtTime(99));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        rleSparseVector.getCapacityAtTime(start + alloc.length + 1));
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (alloc[i] + i), (alloc[i] + i)),
+          rleSparseVector.getCapacityAtTime(start + i));
+    }
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
+    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+      rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
+    }
+    LOG.info(rleSparseVector.toString());
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(Resource.newInstance(0, 0),
+          rleSparseVector.getCapacityAtTime(start + i));
+    }
+    Assert.assertTrue(rleSparseVector.isEmpty());
+  }
+
+  @Test
+  public void testSkyline() {
+    ResourceCalculator resCalc = new DefaultResourceCalculator();
+    Resource minAlloc = Resource.newInstance(1, 1);
+
+    RLESparseResourceAllocation rleSparseVector =
+        new RLESparseResourceAllocation(resCalc, minAlloc);
+    int[] alloc = { 0, 5, 10, 10, 5, 0 };
+    int start = 100;
+    Set<Entry<ReservationInterval, ReservationRequest>> inputs =
+        generateAllocation(start, alloc, true).entrySet();
+    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+      rleSparseVector.addInterval(ip.getKey(), ip.getValue());
+    }
+    LOG.info(rleSparseVector.toString());
+    Assert.assertFalse(rleSparseVector.isEmpty());
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        rleSparseVector.getCapacityAtTime(99));
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        rleSparseVector.getCapacityAtTime(start + alloc.length + 1));
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(
+          Resource.newInstance(1024 * (alloc[i] + i), (alloc[i] + i)),
+          rleSparseVector.getCapacityAtTime(start + i));
+    }
+    Assert.assertEquals(Resource.newInstance(0, 0),
+        rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
+    for (Entry<ReservationInterval, ReservationRequest> ip : inputs) {
+      rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
+    }
+    LOG.info(rleSparseVector.toString());
+    for (int i = 0; i < alloc.length; i++) {
+      Assert.assertEquals(Resource.newInstance(0, 0),
+          rleSparseVector.getCapacityAtTime(start + i));
+    }
+    Assert.assertTrue(rleSparseVector.isEmpty());
+  }
+
+  @Test
+  public void testZeroAlloaction() {
+    ResourceCalculator resCalc = new DefaultResourceCalculator();
+    Resource minAlloc = Resource.newInstance(1, 1);
+    RLESparseResourceAllocation rleSparseVector =
+        new RLESparseResourceAllocation(resCalc, minAlloc);
+    rleSparseVector.addInterval(new ReservationInterval(0, Long.MAX_VALUE),
+        ReservationRequest.newInstance(Resource.newInstance(0, 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(
+      int startTime, int[] alloc, boolean isStep) {
+    Map<ReservationInterval, ReservationRequest> req =
+        new HashMap<ReservationInterval, ReservationRequest>();
+    int numContainers = 0;
+    for (int i = 0; i < alloc.length; i++) {
+      if (isStep) {
+        numContainers = alloc[i] + i;
+      } else {
+        numContainers = alloc[i];
+      }
+      req.put(new ReservationInterval(startTime + i, startTime + i + 1),
+
+      ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+          (numContainers)));
+    }
+    return req;
+  }
+
+}


[03/16] YARN-1708. Public YARN APIs for creating/updating/deleting reservations. (cherry picked from commit 3f2e3b275bcf29264a112c5d2f4eae289fe7f52c)

Posted by cd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionRequestPBImpl.java
new file mode 100644
index 0000000..9f10f4c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionRequestPBImpl.java
@@ -0,0 +1,152 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationDefinitionPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationDefinitionProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionRequestProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+public class ReservationSubmissionRequestPBImpl extends
+    ReservationSubmissionRequest {
+
+  ReservationSubmissionRequestProto proto = ReservationSubmissionRequestProto
+      .getDefaultInstance();
+  ReservationSubmissionRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ReservationDefinition reservationDefinition;
+
+  public ReservationSubmissionRequestPBImpl() {
+    builder = ReservationSubmissionRequestProto.newBuilder();
+  }
+
+  public ReservationSubmissionRequestPBImpl(
+      ReservationSubmissionRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationSubmissionRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationDefinition != null) {
+      builder
+          .setReservationDefinition(convertToProtoFormat(reservationDefinition));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationSubmissionRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public ReservationDefinition getReservationDefinition() {
+    ReservationSubmissionRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationDefinition != null) {
+      return reservationDefinition;
+    }
+    if (!p.hasReservationDefinition()) {
+      return null;
+    }
+    reservationDefinition =
+        convertFromProtoFormat(p.getReservationDefinition());
+    return reservationDefinition;
+  }
+
+  @Override
+  public void setReservationDefinition(
+      ReservationDefinition reservationDefinition) {
+    maybeInitBuilder();
+    if (reservationDefinition == null) {
+      builder.clearReservationDefinition();
+    }
+    this.reservationDefinition = reservationDefinition;
+  }
+
+  @Override
+  public String getQueue() {
+    ReservationSubmissionRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasQueue()) {
+      return null;
+    }
+    return (p.getQueue());
+  }
+
+  @Override
+  public void setQueue(String planName) {
+    maybeInitBuilder();
+    if (planName == null) {
+      builder.clearQueue();
+      return;
+    }
+    builder.setQueue(planName);
+  }
+
+  private ReservationDefinitionProto convertToProtoFormat(
+      ReservationDefinition r) {
+    return ((ReservationDefinitionPBImpl) r).getProto();
+  }
+
+  private ReservationDefinitionPBImpl convertFromProtoFormat(
+      ReservationDefinitionProto r) {
+    return new ReservationDefinitionPBImpl(r);
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionResponsePBImpl.java
new file mode 100644
index 0000000..82151f4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionResponsePBImpl.java
@@ -0,0 +1,129 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationIdPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationIdProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionResponseProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+public class ReservationSubmissionResponsePBImpl extends
+    ReservationSubmissionResponse {
+
+  ReservationSubmissionResponseProto proto = ReservationSubmissionResponseProto
+      .getDefaultInstance();
+  ReservationSubmissionResponseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ReservationId reservationId;
+
+  public ReservationSubmissionResponsePBImpl() {
+    builder = ReservationSubmissionResponseProto.newBuilder();
+  }
+
+  public ReservationSubmissionResponsePBImpl(
+      ReservationSubmissionResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationSubmissionResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationId != null) {
+      builder.setReservationId(convertToProtoFormat(this.reservationId));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationSubmissionResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public ReservationId getReservationId() {
+    ReservationSubmissionResponseProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationId != null) {
+      return reservationId;
+    }
+    if (!p.hasReservationId()) {
+      return null;
+    }
+    reservationId = convertFromProtoFormat(p.getReservationId());
+    return reservationId;
+  }
+
+  @Override
+  public void setReservationId(ReservationId reservationId) {
+    maybeInitBuilder();
+    if (reservationId == null) {
+      builder.clearReservationId();
+      return;
+    }
+    this.reservationId = reservationId;
+  }
+
+  private ReservationIdPBImpl convertFromProtoFormat(ReservationIdProto p) {
+    return new ReservationIdPBImpl(p);
+  }
+
+  private ReservationIdProto convertToProtoFormat(ReservationId t) {
+    return ((ReservationIdPBImpl) t).getProto();
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationUpdateRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationUpdateRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationUpdateRequestPBImpl.java
new file mode 100644
index 0000000..89999f2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationUpdateRequestPBImpl.java
@@ -0,0 +1,169 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationDefinitionPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationIdPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationDefinitionProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationIdProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateRequestProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+public class ReservationUpdateRequestPBImpl extends ReservationUpdateRequest {
+
+  ReservationUpdateRequestProto proto = ReservationUpdateRequestProto
+      .getDefaultInstance();
+  ReservationUpdateRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ReservationDefinition reservationDefinition;
+  private ReservationId reservationId;
+
+  public ReservationUpdateRequestPBImpl() {
+    builder = ReservationUpdateRequestProto.newBuilder();
+  }
+
+  public ReservationUpdateRequestPBImpl(ReservationUpdateRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationUpdateRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationId != null) {
+      builder.setReservationId(convertToProtoFormat(this.reservationId));
+    }
+    if (this.reservationDefinition != null) {
+      builder
+          .setReservationDefinition(convertToProtoFormat(reservationDefinition));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationUpdateRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public ReservationDefinition getReservationDefinition() {
+    ReservationUpdateRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationDefinition != null) {
+      return reservationDefinition;
+    }
+    if (!p.hasReservationDefinition()) {
+      return null;
+    }
+    reservationDefinition =
+        convertFromProtoFormat(p.getReservationDefinition());
+    return reservationDefinition;
+  }
+
+  @Override
+  public void setReservationDefinition(
+      ReservationDefinition reservationDefinition) {
+    maybeInitBuilder();
+    if (reservationDefinition == null) {
+      builder.clearReservationDefinition();
+    }
+    this.reservationDefinition = reservationDefinition;
+  }
+
+  @Override
+  public ReservationId getReservationId() {
+    ReservationUpdateRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationId != null) {
+      return reservationId;
+    }
+    if (!p.hasReservationId()) {
+      return null;
+    }
+    reservationId = convertFromProtoFormat(p.getReservationId());
+    return reservationId;
+  }
+
+  @Override
+  public void setReservationId(ReservationId reservationId) {
+    maybeInitBuilder();
+    if (reservationId == null) {
+      builder.clearReservationId();
+      return;
+    }
+    this.reservationId = reservationId;
+  }
+
+  private ReservationIdPBImpl convertFromProtoFormat(ReservationIdProto p) {
+    return new ReservationIdPBImpl(p);
+  }
+
+  private ReservationIdProto convertToProtoFormat(ReservationId t) {
+    return ((ReservationIdPBImpl) t).getProto();
+  }
+
+  private ReservationDefinitionProto convertToProtoFormat(
+      ReservationDefinition r) {
+    return ((ReservationDefinitionPBImpl) r).getProto();
+  }
+
+  private ReservationDefinitionPBImpl convertFromProtoFormat(
+      ReservationDefinitionProto r) {
+    return new ReservationDefinitionPBImpl(r);
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationUpdateResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationUpdateResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationUpdateResponsePBImpl.java
new file mode 100644
index 0000000..57c7b1b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationUpdateResponsePBImpl.java
@@ -0,0 +1,68 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateResponseProto;
+
+import com.google.protobuf.TextFormat;
+
+public class ReservationUpdateResponsePBImpl extends ReservationUpdateResponse {
+
+  ReservationUpdateResponseProto proto = ReservationUpdateResponseProto
+      .getDefaultInstance();
+  ReservationUpdateResponseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  public ReservationUpdateResponsePBImpl() {
+    builder = ReservationUpdateResponseProto.newBuilder();
+  }
+
+  public ReservationUpdateResponsePBImpl(ReservationUpdateResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationUpdateResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
index e4f183b..489cf8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
@@ -57,6 +57,7 @@ extends ApplicationSubmissionContext {
   private Resource resource = null;
   private Set<String> applicationTags = null;
   private LogAggregationContext logAggregationContext = null;
+  private ReservationId reservationId = null;
 
   public ApplicationSubmissionContextPBImpl() {
     builder = ApplicationSubmissionContextProto.newBuilder();
@@ -114,10 +115,6 @@ extends ApplicationSubmissionContext {
       builder.clearApplicationTags();
       builder.addAllApplicationTags(this.applicationTags);
     }
-    if (this.logAggregationContext != null) {
-      builder.setLogAggregationContext(
-          convertToProtoFormat(this.logAggregationContext));
-    }
   }
 
   private void mergeLocalToProto() {
@@ -366,6 +363,29 @@ extends ApplicationSubmissionContext {
   }
 
   @Override
+  public ReservationId getReservationID() {
+    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationId != null) {
+      return reservationId;
+    }
+    if (!p.hasReservationId()) {
+      return null;
+    }
+    reservationId = convertFromProtoFormat(p.getReservationId());
+    return reservationId;
+  }
+
+  @Override
+  public void setReservationID(ReservationId reservationID) {
+    maybeInitBuilder();
+    if (reservationID == null) {
+      builder.clearReservationId();
+      return;
+    }
+    this.reservationId = reservationID;
+  }
+
+  @Override
   public void
       setKeepContainersAcrossApplicationAttempts(boolean keepContainers) {
     maybeInitBuilder();
@@ -455,4 +475,12 @@ extends ApplicationSubmissionContext {
       builder.clearLogAggregationContext();
     this.logAggregationContext = logAggregationContext;
   }
+
+  private ReservationIdPBImpl convertFromProtoFormat(ReservationIdProto p) {
+    return new ReservationIdPBImpl(p);
+  }
+
+  private ReservationIdProto convertToProtoFormat(ReservationId t) {
+    return ((ReservationIdPBImpl) t).getProto();
+  }
 }  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
index 4a3c137..f18a77c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueState;
+import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.proto.YarnProtos.AMCommandProto;
@@ -47,10 +48,12 @@ import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueACLProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueStateProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestInterpreterProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationAttemptStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
 
 import com.google.protobuf.ByteString;
+
 import org.apache.hadoop.yarn.proto.YarnServiceProtos;
 
 @Private
@@ -237,4 +240,18 @@ public class ProtoUtils {
     return ApplicationAccessType.valueOf(e.name().replace(
         APP_ACCESS_TYPE_PREFIX, ""));
   }
+  
+  /*
+   * Reservation Request interpreter type
+   */
+  public static ReservationRequestInterpreterProto convertToProtoFormat(
+      ReservationRequestInterpreter e) {
+    return ReservationRequestInterpreterProto.valueOf(e.name());
+  }
+
+  public static ReservationRequestInterpreter convertFromProtoFormat(
+      ReservationRequestInterpreterProto e) {
+    return ReservationRequestInterpreter.valueOf(e.name());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java
new file mode 100644
index 0000000..9f87108
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java
@@ -0,0 +1,169 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationDefinitionProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationDefinitionProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestsProto;
+
+public class ReservationDefinitionPBImpl extends ReservationDefinition {
+
+  ReservationDefinitionProto proto = ReservationDefinitionProto
+      .getDefaultInstance();
+  ReservationDefinitionProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ReservationRequests reservationReqs;
+
+  public ReservationDefinitionPBImpl() {
+    builder = ReservationDefinitionProto.newBuilder();
+  }
+
+  public ReservationDefinitionPBImpl(ReservationDefinitionProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationDefinitionProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationReqs != null) {
+      builder
+          .setReservationRequests(convertToProtoFormat(this.reservationReqs));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationDefinitionProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public long getArrival() {
+    ReservationDefinitionProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasArrival()) {
+      return 0;
+    }
+    return (p.getArrival());
+  }
+
+  @Override
+  public void setArrival(long earliestStartTime) {
+    maybeInitBuilder();
+    if (earliestStartTime <= 0) {
+      builder.clearArrival();
+      return;
+    }
+    builder.setArrival(earliestStartTime);
+  }
+
+  @Override
+  public long getDeadline() {
+    ReservationDefinitionProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasDeadline()) {
+      return 0;
+    }
+    return (p.getDeadline());
+  }
+
+  @Override
+  public void setDeadline(long latestEndTime) {
+    maybeInitBuilder();
+    if (latestEndTime <= 0) {
+      builder.clearDeadline();
+      return;
+    }
+    builder.setDeadline(latestEndTime);
+  }
+
+  @Override
+  public ReservationRequests getReservationRequests() {
+    ReservationDefinitionProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationReqs != null) {
+      return reservationReqs;
+    }
+    if (!p.hasReservationRequests()) {
+      return null;
+    }
+    reservationReqs = convertFromProtoFormat(p.getReservationRequests());
+    return reservationReqs;
+  }
+
+  @Override
+  public void setReservationRequests(ReservationRequests reservationRequests) {
+    if (reservationRequests == null) {
+      builder.clearReservationRequests();
+      return;
+    }
+    this.reservationReqs = reservationRequests;
+  }
+
+  @Override
+  public String getReservationName() {
+    ReservationDefinitionProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasReservationName()) {
+      return null;
+    }
+    return (p.getReservationName());
+  }
+
+  @Override
+  public void setReservationName(String name) {
+    maybeInitBuilder();
+    if (name == null) {
+      builder.clearReservationName();
+      return;
+    }
+    builder.setReservationName(name);
+  }
+
+  private ReservationRequestsPBImpl convertFromProtoFormat(
+      ReservationRequestsProto p) {
+    return new ReservationRequestsPBImpl(p);
+  }
+
+  private ReservationRequestsProto convertToProtoFormat(ReservationRequests t) {
+    return ((ReservationRequestsPBImpl) t).getProto();
+  }
+
+  @Override
+  public String toString() {
+    return "{Arrival: " + getArrival() + ", Deadline: " + getDeadline()
+        + ", Reservation Name: " + getReservationName() + ", Resources: "
+        + getReservationRequests() + "}";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationIdPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationIdPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationIdPBImpl.java
new file mode 100644
index 0000000..609d4ab
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationIdPBImpl.java
@@ -0,0 +1,75 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationIdProto;
+
+import com.google.common.base.Preconditions;
+
+@Private
+@Unstable
+public class ReservationIdPBImpl extends ReservationId {
+  ReservationIdProto proto = null;
+  ReservationIdProto.Builder builder = null;
+
+  public ReservationIdPBImpl() {
+    builder = ReservationIdProto.newBuilder();
+  }
+
+  public ReservationIdPBImpl(ReservationIdProto proto) {
+    this.proto = proto;
+  }
+
+  public ReservationIdProto getProto() {
+    return proto;
+  }
+
+  @Override
+  public long getId() {
+    Preconditions.checkNotNull(proto);
+    return proto.getId();
+  }
+
+  @Override
+  protected void setId(long id) {
+    Preconditions.checkNotNull(builder);
+    builder.setId(id);
+  }
+
+  @Override
+  public long getClusterTimestamp() {
+    Preconditions.checkNotNull(proto);
+    return proto.getClusterTimestamp();
+  }
+
+  @Override
+  protected void setClusterTimestamp(long clusterTimestamp) {
+    Preconditions.checkNotNull(builder);
+    builder.setClusterTimestamp((clusterTimestamp));
+  }
+
+  @Override
+  protected void build() {
+    proto = builder.build();
+    builder = null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestPBImpl.java
new file mode 100644
index 0000000..e36b9fb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestPBImpl.java
@@ -0,0 +1,152 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+
+@Private
+@Unstable
+public class ReservationRequestPBImpl extends ReservationRequest {
+  ReservationRequestProto proto = ReservationRequestProto.getDefaultInstance();
+  ReservationRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private Resource capability = null;
+
+  public ReservationRequestPBImpl() {
+    builder = ReservationRequestProto.newBuilder();
+  }
+
+  public ReservationRequestPBImpl(ReservationRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.capability != null) {
+      builder.setCapability(convertToProtoFormat(this.capability));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public Resource getCapability() {
+    ReservationRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.capability != null) {
+      return this.capability;
+    }
+    if (!p.hasCapability()) {
+      return null;
+    }
+    this.capability = convertFromProtoFormat(p.getCapability());
+    return this.capability;
+  }
+
+  @Override
+  public void setCapability(Resource capability) {
+    maybeInitBuilder();
+    if (capability == null)
+      builder.clearCapability();
+    this.capability = capability;
+  }
+
+  @Override
+  public int getNumContainers() {
+    ReservationRequestProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getNumContainers());
+  }
+
+  @Override
+  public void setNumContainers(int numContainers) {
+    maybeInitBuilder();
+    builder.setNumContainers((numContainers));
+  }
+
+  @Override
+  public int getConcurrency() {
+    ReservationRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasConcurrency()) {
+      return 1;
+    }
+    return (p.getConcurrency());
+  }
+
+  @Override
+  public void setConcurrency(int numContainers) {
+    maybeInitBuilder();
+    builder.setConcurrency(numContainers);
+  }
+
+  @Override
+  public long getDuration() {
+    ReservationRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasDuration()) {
+      return 0;
+    }
+    return (p.getDuration());
+  }
+
+  @Override
+  public void setDuration(long duration) {
+    maybeInitBuilder();
+    builder.setDuration(duration);
+  }
+
+  private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
+    return new ResourcePBImpl(p);
+  }
+
+  private ResourceProto convertToProtoFormat(Resource t) {
+    return ((ResourcePBImpl) t).getProto();
+  }
+
+  @Override
+  public String toString() {
+    return "{Capability: " + getCapability() + ", # Containers: "
+        + getNumContainers() + ", Concurrency: " + getConcurrency()
+        + ", Lease Duration: " + getDuration() + "}";
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestsPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestsPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestsPBImpl.java
new file mode 100644
index 0000000..4b39836
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestsPBImpl.java
@@ -0,0 +1,189 @@
+/**
+ * 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.api.records.impl.pb;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
+import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestInterpreterProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestsProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestsProtoOrBuilder;
+
+public class ReservationRequestsPBImpl extends ReservationRequests {
+
+  ReservationRequestsProto proto = ReservationRequestsProto
+      .getDefaultInstance();
+  ReservationRequestsProto.Builder builder = null;
+  boolean viaProto = false;
+
+  public List<ReservationRequest> reservationRequests;
+
+  public ReservationRequestsPBImpl() {
+    builder = ReservationRequestsProto.newBuilder();
+  }
+
+  public ReservationRequestsPBImpl(ReservationRequestsProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationRequestsProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationRequests != null) {
+      addReservationResourcesToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationRequestsProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public List<ReservationRequest> getReservationResources() {
+    initReservationRequestsList();
+    return reservationRequests;
+  }
+
+  @Override
+  public void setReservationResources(List<ReservationRequest> resources) {
+    if (resources == null) {
+      builder.clearReservationResources();
+      return;
+    }
+    this.reservationRequests = resources;
+  }
+
+  @Override
+  public ReservationRequestInterpreter getInterpreter() {
+    ReservationRequestsProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasInterpreter()) {
+      return null;
+    }
+    return (convertFromProtoFormat(p.getInterpreter()));
+  }
+
+  @Override
+  public void setInterpreter(ReservationRequestInterpreter interpreter) {
+    maybeInitBuilder();
+    if (interpreter == null) {
+      builder.clearInterpreter();
+      return;
+    }
+    builder.setInterpreter(convertToProtoFormat(interpreter));
+  }
+
+  private void initReservationRequestsList() {
+    if (this.reservationRequests != null) {
+      return;
+    }
+    ReservationRequestsProtoOrBuilder p = viaProto ? proto : builder;
+    List<ReservationRequestProto> resourceProtos =
+        p.getReservationResourcesList();
+    reservationRequests = new ArrayList<ReservationRequest>();
+
+    for (ReservationRequestProto r : resourceProtos) {
+      reservationRequests.add(convertFromProtoFormat(r));
+    }
+  }
+
+  private void addReservationResourcesToProto() {
+    maybeInitBuilder();
+    builder.clearReservationResources();
+    if (reservationRequests == null)
+      return;
+    Iterable<ReservationRequestProto> iterable =
+        new Iterable<ReservationRequestProto>() {
+          @Override
+          public Iterator<ReservationRequestProto> iterator() {
+            return new Iterator<ReservationRequestProto>() {
+
+              Iterator<ReservationRequest> iter = reservationRequests
+                  .iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iter.hasNext();
+              }
+
+              @Override
+              public ReservationRequestProto next() {
+                return convertToProtoFormat(iter.next());
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+
+            };
+
+          }
+
+        };
+    builder.addAllReservationResources(iterable);
+  }
+
+  private ReservationRequestProto convertToProtoFormat(ReservationRequest r) {
+    return ((ReservationRequestPBImpl) r).getProto();
+  }
+
+  private ReservationRequestPBImpl convertFromProtoFormat(
+      ReservationRequestProto r) {
+    return new ReservationRequestPBImpl(r);
+  }
+
+  private ReservationRequestInterpreterProto convertToProtoFormat(
+      ReservationRequestInterpreter r) {
+    return ProtoUtils.convertToProtoFormat(r);
+  }
+
+  private ReservationRequestInterpreter convertFromProtoFormat(
+      ReservationRequestInterpreterProto r) {
+    return ProtoUtils.convertFromProtoFormat(r);
+  }
+
+  @Override
+  public String toString() {
+    return "{Reservation Resources: " + getReservationResources()
+        + ", Reservation Type: " + getInterpreter() + "}";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/UTCClock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/UTCClock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/UTCClock.java
new file mode 100644
index 0000000..ecef3dd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/UTCClock.java
@@ -0,0 +1,39 @@
+/**
+ * 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.util;
+
+import java.util.Calendar;
+import java.util.TimeZone;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+/**
+ * Implementation of {@link Clock} that gives the current UTC time in
+ * milliseconds.
+ */
+@Public
+@Evolving
+public class UTCClock implements Clock {
+
+  private final TimeZone utcZone = TimeZone.getTimeZone("UTC");
+
+  public long getTime() {
+    return Calendar.getInstance(utcZone).getTimeInMillis();
+  }
+}
\ No newline at end of file


[14/16] git commit: YARN-2576. Making test patch pass in branch. Contributed by Subru Krishnan and Carlo Curino. (cherry picked from commit 90ac0be86b898aefec5471db4027554c8e1b310c)

Posted by cd...@apache.org.
YARN-2576. Making test patch pass in branch. Contributed by Subru Krishnan and Carlo Curino.
(cherry picked from commit 90ac0be86b898aefec5471db4027554c8e1b310c)


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

Branch: refs/heads/trunk
Commit: 5e10a13bb4759984494c6a870c7f08fb6693c9c0
Parents: 3f28276
Author: subru <su...@outlook.com>
Authored: Wed Sep 24 18:13:21 2014 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Oct 3 15:43:13 2014 -0700

----------------------------------------------------------------------
 YARN-1051-CHANGES.txt                           |  3 ++
 .../yarn/api/ApplicationClientProtocol.java     |  8 ++--
 .../ReservationDeleteResponse.java              |  2 +-
 .../ReservationSubmissionRequest.java           | 12 ++---
 .../ReservationSubmissionResponse.java          |  4 +-
 .../ReservationUpdateResponse.java              |  2 +-
 .../records/ApplicationSubmissionContext.java   |  4 +-
 .../yarn/api/records/ReservationDefinition.java |  2 +-
 .../hadoop/yarn/api/records/ReservationId.java  | 37 +++++++++++++--
 .../yarn/api/records/ReservationRequest.java    |  2 +-
 .../records/ReservationRequestInterpreter.java  | 20 +++++++-
 .../yarn/api/records/ReservationRequests.java   |  2 +-
 .../hadoop/yarn/client/api/YarnClient.java      | 43 ++++++++++--------
 .../impl/pb/ReservationDefinitionPBImpl.java    | 15 ++++++
 .../impl/pb/ReservationRequestsPBImpl.java      | 15 ++++++
 .../hadoop/yarn/api/TestPBImplRecords.java      | 41 +++++++++++++++++
 .../reservation/AbstractReservationSystem.java  | 15 ++++++
 .../reservation/CapacityOverTimePolicy.java     | 17 +++++++
 .../reservation/CapacityReservationSystem.java  | 15 ++++++
 .../reservation/GreedyReservationAgent.java     | 18 ++++++++
 .../reservation/InMemoryPlan.java               | 21 +++++++++
 .../InMemoryReservationAllocation.java          | 17 +++++++
 .../reservation/NoOverCommitPolicy.java         | 20 +++++++-
 .../resourcemanager/reservation/Plan.java       | 20 ++++++++
 .../reservation/PlanContext.java                | 18 ++++++++
 .../resourcemanager/reservation/PlanEdit.java   | 22 ++++++++-
 .../reservation/PlanFollower.java               | 17 +++++++
 .../resourcemanager/reservation/PlanView.java   | 17 +++++++
 .../resourcemanager/reservation/Planner.java    | 48 ++++++++++++++++++++
 .../RLESparseResourceAllocation.java            | 18 ++++++++
 .../reservation/ReservationAgent.java           | 17 +++++++
 .../reservation/ReservationAllocation.java      | 20 +++++++-
 .../reservation/ReservationInputValidator.java  | 18 ++++++++
 .../reservation/ReservationInterval.java        | 18 ++++++++
 .../reservation/SharingPolicy.java              | 22 ++++++++-
 .../reservation/SimpleCapacityReplanner.java    | 25 +++++++++-
 .../exceptions/ContractValidationException.java | 18 ++++++++
 .../exceptions/MismatchedUserException.java     | 18 ++++++++
 .../exceptions/PlanningException.java           | 18 ++++++++
 .../exceptions/PlanningQuotaException.java      | 18 ++++++++
 .../exceptions/ResourceOverCommitException.java | 19 ++++++++
 .../SchedulerDynamicEditException.java          | 18 ++++++++
 .../scheduler/capacity/CapacityScheduler.java   | 11 ++++-
 .../CapacitySchedulerConfiguration.java         |  2 +-
 .../scheduler/capacity/PlanQueue.java           | 18 ++++++++
 .../scheduler/capacity/ReservationQueue.java    | 18 ++++++++
 .../scheduler/common/QueueEntitlement.java      | 18 ++++++++
 .../webapp/dao/CapacitySchedulerQueueInfo.java  |  6 +--
 .../applicationsmanager/MockAsm.java            |  6 +++
 .../reservation/ReservationSystemTestUtil.java  | 23 ++++++++--
 .../reservation/TestCapacityOverTimePolicy.java | 17 +++++++
 .../TestCapacityReservationSystem.java          | 17 +++++++
 .../reservation/TestGreedyReservationAgent.java | 22 +++++++--
 .../reservation/TestInMemoryPlan.java           | 17 +++++++
 .../TestInMemoryReservationAllocation.java      | 17 +++++++
 .../reservation/TestNoOverCommitPolicy.java     | 17 +++++++
 .../TestRLESparseResourceAllocation.java        | 17 +++++++
 .../TestReservationInputValidator.java          | 17 +++++++
 .../TestSimpleCapacityReplanner.java            | 20 +++++++-
 .../server/resourcemanager/rmapp/MockRMApp.java |  6 +++
 .../TestCapacitySchedulerDynamicBehavior.java   |  4 +-
 61 files changed, 929 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/YARN-1051-CHANGES.txt
----------------------------------------------------------------------
diff --git a/YARN-1051-CHANGES.txt b/YARN-1051-CHANGES.txt
index 6a27197..7620cc3 100644
--- a/YARN-1051-CHANGES.txt
+++ b/YARN-1051-CHANGES.txt
@@ -26,3 +26,6 @@ client-RM protocol. (Subru Krishnan and Carlo Curino  via subru)
 
 MAPREDUCE-6103. Adding reservation APIs to MR resource manager
 delegate. (Subru Krishnan and Carlo Curino  via subru)
+
+YARN-2576. Fixing compilation, javadocs and audit issues to pass
+test patch in branch. (Subru Krishnan and Carlo Curino  via subru)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
index 863a068..e44ddb3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
@@ -553,7 +553,7 @@ public interface ApplicationClientProtocol {
   /**
    * <p>
    * The interface used by clients to submit a new reservation to the
-   * {@link ResourceManager}.
+   * {@code ResourceManager}.
    * </p>
    * 
    * <p>
@@ -566,12 +566,12 @@ public interface ApplicationClientProtocol {
    * 
    * <p>
    * In order to respond, a new admission control component in the
-   * {@link ResourceManager} performs an analysis of the resources that have
+   * {@code ResourceManager} performs an analysis of the resources that have
    * been committed over the period of time the user is requesting, verify that
    * the user requests can be fulfilled, and that it respect a sharing policy
-   * (e.g., {@link CapacityOverTimePolicy}). Once it has positively determined
+   * (e.g., {@code CapacityOverTimePolicy}). Once it has positively determined
    * that the ReservationSubmissionRequest is satisfiable the
-   * {@link ResourceManager} answers with a
+   * {@code ResourceManager} answers with a
    * {@link ReservationSubmissionResponse} that include a non-null
    * {@link ReservationId}. Upon failure to find a valid allocation the response
    * is an exception with the reason.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteResponse.java
index eedd3a4..085f0d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteResponse.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.yarn.util.Records;
 
 /**
  * {@link ReservationDeleteResponse} contains the answer of the admission
- * control system in the {@link ResourceManager} to a reservation delete
+ * control system in the {@code ResourceManager} to a reservation delete
  * operation. Currently response is empty if the operation was successful, if
  * not an exception reporting reason for a failure.
  * 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java
index e550fe2..6d5ca16 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java
@@ -69,11 +69,11 @@ public abstract class ReservationSubmissionRequest {
       ReservationDefinition reservationDefinition);
 
   /**
-   * Get the name of the {@link Plan} that corresponds to the name of the
+   * Get the name of the {@code Plan} that corresponds to the name of the
    * {@link QueueInfo} in the scheduler to which the reservation will be
    * submitted to.
    * 
-   * @return the name of the {@link Plan} that corresponds to the name of the
+   * @return the name of the {@code Plan} that corresponds to the name of the
    *         {@link QueueInfo} in the scheduler to which the reservation will be
    *         submitted to
    */
@@ -82,13 +82,13 @@ public abstract class ReservationSubmissionRequest {
   public abstract String getQueue();
 
   /**
-   * Set the name of the {@link Plan} that corresponds to the name of the
+   * Set the name of the {@code Plan} that corresponds to the name of the
    * {@link QueueInfo} in the scheduler to which the reservation will be
    * submitted to
    * 
-   * @param the name of the parent {@link Plan} that corresponds to the name of
-   *          the {@link QueueInfo} in the scheduler to which the reservation
-   *          will be submitted to
+   * @param queueName the name of the parent {@code Plan} that corresponds to
+   *          the name of the {@link QueueInfo} in the scheduler to which the
+   *          reservation will be submitted to
    */
   @Public
   @Unstable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java
index b57ef52..32fe5e0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java
@@ -25,11 +25,9 @@ import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.util.Records;
 
-import com.sun.naming.internal.ResourceManager;
-
 /**
  * {@link ReservationSubmissionResponse} contains the answer of the admission
- * control system in the {@link ResourceManager} to a reservation create
+ * control system in the {@code ResourceManager} to a reservation create
  * operation. Response contains a {@link ReservationId} if the operation was
  * successful, if not an exception reporting reason for a failure.
  * 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateResponse.java
index bfd2d51..1b5c195 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateResponse.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.yarn.util.Records;
 
 /**
  * {@link ReservationUpdateResponse} contains the answer of the admission
- * control system in the {@link ResourceManager} to a reservation update
+ * control system in the {@code ResourceManager} to a reservation update
  * operation. Currently response is empty if the operation was successful, if
  * not an exception reporting reason for a failure.
  * 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
index cbf1e37..74da4b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
@@ -433,10 +433,10 @@ public abstract class ApplicationSubmissionContext {
    * Set the reservation id, that correspond to a valid resource allocation in
    * the scheduler (between start and end time of the corresponding reservation)
    * 
-   * @param reservationId representing the unique id of the
+   * @param reservationID representing the unique id of the
    *          corresponding reserved resource allocation in the scheduler
    */
   @Public
   @Unstable
   public abstract void setReservationID(ReservationId reservationID);
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java
index c94463d..10f592a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java
@@ -103,7 +103,7 @@ public abstract class ReservationDefinition {
    * Set the list of {@link ReservationRequests} representing the resources
    * required by the application
    * 
-   * @param resources the list of {@link ReservationRequests}
+   * @param reservationRequests the list of {@link ReservationRequests}
    */
   @Public
   @Unstable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
index 47a8c40..71474b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.api.records;
 
+import java.io.IOException;
 import java.text.NumberFormat;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -33,7 +34,7 @@ import org.apache.hadoop.yarn.util.Records;
  * 
  * <p>
  * The globally unique nature of the identifier is achieved by using the
- * <em>cluster timestamp</em> i.e. start-time of the {@link ResourceManager}
+ * <em>cluster timestamp</em> i.e. start-time of the {@code ResourceManager}
  * along with a monotonically increasing counter for the reservation.
  * </p>
  */
@@ -60,7 +61,7 @@ public abstract class ReservationId implements Comparable<ReservationId> {
   /**
    * Get the long identifier of the {@link ReservationId} which is unique for
    * all Reservations started by a particular instance of the
-   * {@link ResourceManager}.
+   * {@code ResourceManager}.
    * 
    * @return long identifier of the {@link ReservationId}
    */
@@ -73,10 +74,10 @@ public abstract class ReservationId implements Comparable<ReservationId> {
   protected abstract void setId(long id);
 
   /**
-   * Get the <em>start time</em> of the {@link ResourceManager} which is used to
+   * Get the <em>start time</em> of the {@code ResourceManager} which is used to
    * generate globally unique {@link ReservationId}.
    * 
-   * @return <em>start time</em> of the {@link ResourceManager}
+   * @return <em>start time</em> of the {@code ResourceManager}
    */
   @Public
   @Unstable
@@ -115,6 +116,34 @@ public abstract class ReservationId implements Comparable<ReservationId> {
         + reservIdFormat.get().format(getId());
   }
 
+  /**
+   * Parse the string argument as a {@link ReservationId}
+   *
+   * @param reservationId the string representation of the {@link ReservationId}
+   * @return the {@link ReservationId} corresponding to the input string if
+   *         valid, null if input is null
+   * @throws IOException if unable to parse the input string
+   */
+  @Public
+  @Unstable
+  public static ReservationId parseReservationId(String reservationId)
+      throws IOException {
+    if (reservationId == null) {
+      return null;
+    }
+    if (!reservationId.startsWith(reserveIdStrPrefix)) {
+      throw new IOException("The specified reservation id is invalid: "
+          + reservationId);
+    }
+    String[] resFields = reservationId.split("_");
+    if (resFields.length != 3) {
+      throw new IOException("The specified reservation id is not parseable: "
+          + reservationId);
+    }
+    return newInstance(Long.parseLong(resFields[1]),
+        Long.parseLong(resFields[2]));
+  }
+
   @Override
   public int hashCode() {
     // generated by eclipse

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
index ded6d06..a4f0585 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.yarn.util.Records;
 /**
  * <p>
  * {@link ReservationRequest} represents the request made by an application to
- * the {@link ResourceManager} to reserve {@link Resource}s.
+ * the {@code ResourceManager} to reserve {@link Resource}s.
  * </p>
  * 
  * <p>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
index 4621e0d..1ee96c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
@@ -1,3 +1,21 @@
+/**
+ * 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.api.records;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -72,7 +90,7 @@ public enum ReservationRequestInterpreter {
    * {@link ReservationDefinition} are satisfied for the overall
    * {@link ReservationDefinition} to be satisfied. Moreover, it imposes a
    * strict temporal ordering on the allocation used to satisfy the
-   * {@link ResourceRequeust}s. It imposes a strict temporal ordering on the
+   * {@link ReservationRequest}s. It imposes a strict temporal ordering on the
    * allocation used to satisfy the {@link ReservationRequest}s. The allocations
    * satisfying the {@link ReservationRequest} in position k must strictly
    * precede the allocations for the {@link ReservationRequest} at position k+1.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequests.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequests.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequests.java
index 8cc3fbc..80ce8ef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequests.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequests.java
@@ -61,7 +61,7 @@ public abstract class ReservationRequests {
    * Set the list of {@link ReservationRequest} representing the resources
    * required by the application
    * 
-   * @param resources the list of {@link ReservationRequest}
+   * @param reservationResources the list of {@link ReservationRequest}
    */
   @Public
   @Unstable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
index d697de9..7cc6dde 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
@@ -486,36 +487,36 @@ public abstract class YarnClient extends AbstractService {
   /**
    * <p>
    * The interface used by clients to submit a new reservation to the
-   * {@link ResourceManager}.
+   * {@code ResourceManager}.
    * </p>
    * 
    * <p>
    * The client packages all details of its request in a
-   * {@link ReservationRequest} object. This contains information about the
-   * amount of capacity, temporal constraints, and gang needs. Furthermore, the
-   * reservation might be composed of multiple stages, with ordering
-   * dependencies among them.
+   * {@link ReservationSubmissionRequest} object. This contains information
+   * about the amount of capacity, temporal constraints, and gang needs.
+   * Furthermore, the reservation might be composed of multiple stages, with
+   * ordering dependencies among them.
    * </p>
    * 
    * <p>
    * In order to respond, a new admission control component in the
-   * {@link ResourceManager} performs an analysis of the resources that have
+   * {@code ResourceManager} performs an analysis of the resources that have
    * been committed over the period of time the user is requesting, verify that
    * the user requests can be fulfilled, and that it respect a sharing policy
-   * (e.g., {@link CapacityOverTimePolicy}). Once it has positively determined
-   * that the ReservationRequest is satisfiable the {@link ResourceManager}
-   * answers with a {@link ReservationResponse} that include a
+   * (e.g., {@code CapacityOverTimePolicy}). Once it has positively determined
+   * that the ReservationRequest is satisfiable the {@code ResourceManager}
+   * answers with a {@link ReservationSubmissionResponse} that includes a
    * {@link ReservationId}. Upon failure to find a valid allocation the response
    * is an exception with the message detailing the reason of failure.
    * </p>
    * 
    * <p>
-   * The semantics guarantees that the ReservationId returned, corresponds to a
-   * valid reservation existing in the time-range request by the user. The
-   * amount of capacity dedicated to such reservation can vary overtime,
-   * depending of the allocation that has been determined. But it is guaranteed
-   * to satisfy all the constraint expressed by the user in the
-   * {@link ReservationRequest}
+   * The semantics guarantees that the {@link ReservationId} returned,
+   * corresponds to a valid reservation existing in the time-range request by
+   * the user. The amount of capacity dedicated to such reservation can vary
+   * overtime, depending of the allocation that has been determined. But it is
+   * guaranteed to satisfy all the constraint expressed by the user in the
+   * {@link ReservationDefinition}
    * </p>
    * 
    * @param request request to submit a new Reservation
@@ -540,14 +541,15 @@ public abstract class YarnClient extends AbstractService {
    * <p>
    * The allocation is attempted by virtually substituting all previous
    * allocations related to this Reservation with new ones, that satisfy the new
-   * {@link ReservationRequest}. Upon success the previous allocation is
+   * {@link ReservationDefinition}. Upon success the previous allocation is
    * atomically substituted by the new one, and on failure (i.e., if the system
    * cannot find a valid allocation for the updated request), the previous
    * allocation remains valid.
    * </p>
    * 
-   * @param request to update an existing Reservation (the ReservationRequest
-   *          should refer to an existing valid {@link ReservationId})
+   * @param request to update an existing Reservation (the
+   *          {@link ReservationUpdateRequest} should refer to an existing valid
+   *          {@link ReservationId})
    * @return response empty on successfully updating the existing reservation
    * @throws YarnException if the request is invalid or reservation cannot be
    *           updated successfully
@@ -564,8 +566,9 @@ public abstract class YarnClient extends AbstractService {
    * The interface used by clients to remove an existing Reservation.
    * </p>
    * 
-   * @param request to remove an existing Reservation (the ReservationRequest
-   *          should refer to an existing valid {@link ReservationId})
+   * @param request to remove an existing Reservation (the
+   *          {@link ReservationDeleteRequest} should refer to an existing valid
+   *          {@link ReservationId})
    * @return response empty on successfully deleting the existing reservation
    * @throws YarnException if the request is invalid or reservation cannot be
    *           deleted successfully

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java
index 9f87108..cbe7ccd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java
@@ -166,4 +166,19 @@ public class ReservationDefinitionPBImpl extends ReservationDefinition {
         + getReservationRequests() + "}";
   }
 
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestsPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestsPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestsPBImpl.java
index 4b39836..7d26b03 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestsPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestsPBImpl.java
@@ -186,4 +186,19 @@ public class ReservationRequestsPBImpl extends ReservationRequests {
         + ", Reservation Type: " + getInterpreter() + "}";
   }
 
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index c463452..e9ca76f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -216,6 +216,11 @@ public class TestPBImplRecords {
         "root", 1.0f, 1.0f, 0.1f, null, null, QueueState.RUNNING));
     generateByNewInstance(QueueUserACLInfo.class);
     generateByNewInstance(YarnClusterMetrics.class);
+    // for reservation system
+    generateByNewInstance(ReservationId.class);
+    generateByNewInstance(ReservationRequest.class);
+    generateByNewInstance(ReservationRequests.class);
+    generateByNewInstance(ReservationDefinition.class);
   }
 
   private class GetSetPair {
@@ -893,4 +898,40 @@ public class TestPBImplRecords {
     validatePBImplRecord(UpdateNodeResourceResponsePBImpl.class,
         UpdateNodeResourceResponseProto.class);
   }
+
+  @Test
+  public void testReservationSubmissionRequestPBImpl() throws Exception {
+    validatePBImplRecord(ReservationSubmissionRequestPBImpl.class,
+        ReservationSubmissionRequestProto.class);
+  }
+
+  @Test
+  public void testReservationSubmissionResponsePBImpl() throws Exception {
+    validatePBImplRecord(ReservationSubmissionResponsePBImpl.class,
+        ReservationSubmissionResponseProto.class);
+  }
+
+  @Test
+  public void testReservationUpdateRequestPBImpl() throws Exception {
+    validatePBImplRecord(ReservationUpdateRequestPBImpl.class,
+        ReservationUpdateRequestProto.class);
+  }
+
+  @Test
+  public void testReservationUpdateResponsePBImpl() throws Exception {
+    validatePBImplRecord(ReservationUpdateResponsePBImpl.class,
+        ReservationUpdateResponseProto.class);
+  }
+
+  @Test
+  public void testReservationDeleteRequestPBImpl() throws Exception {
+    validatePBImplRecord(ReservationDeleteRequestPBImpl.class,
+        ReservationDeleteRequestProto.class);
+  }
+
+  @Test
+  public void testReservationDeleteResponsePBImpl() throws Exception {
+    validatePBImplRecord(ReservationDeleteResponsePBImpl.class,
+        ReservationDeleteResponseProto.class);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.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/AbstractReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
index f0a9543..91fd12c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
@@ -1,6 +1,21 @@
 /**
+ * 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 java.util.HashMap;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.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/CapacityOverTimePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.java
index 38c0207..680f9ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.java
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 java.util.Date;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityReservationSystem.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/CapacityReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityReservationSystem.java
index 548fde1..9bce0d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityReservationSystem.java
@@ -1,6 +1,21 @@
 /**
+ * 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.classification.InterfaceAudience.LimitedPrivate;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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 3214f93..5a61b94 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
@@ -1,3 +1,21 @@
+/**
+ * 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 java.util.HashMap;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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 99231c4..8394a49 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
@@ -1,3 +1,21 @@
+/**
+ * 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 java.util.ArrayList;
@@ -211,6 +229,9 @@ class InMemoryPlan implements Plan {
         LOG.error(errMsg);
         throw new IllegalArgumentException(errMsg);
       }
+      // validate if we can accept this reservation, throws exception if
+      // validation fails
+      policy.validate(this, reservation);
       if (!removeReservation(currReservation)) {
         LOG.error("Unable to replace reservation: {} from plan.",
             reservation.getReservationId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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 10cc55f..fc8407b 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
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 java.util.Collections;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.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/NoOverCommitPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
index cbe2b78..23f2be4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
@@ -1,3 +1,21 @@
+/**
+ * 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.classification.InterfaceAudience.LimitedPrivate;
@@ -67,7 +85,7 @@ public class NoOverCommitPolicy implements SharingPolicy {
   }
 
   @Override
-  public void init(String inventoryQueuePath, Configuration conf) {
+  public void init(String planQueuePath, Configuration conf) {
     // nothing to do for this policy
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/Plan.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/Plan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/Plan.java
index cf2aed7..e8e9e29 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/Plan.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/Plan.java
@@ -1,5 +1,25 @@
+/**
+ * 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.ReservationDefinition;
+
 /**
  * A Plan represents the central data structure of a reservation system that
  * maintains the "agenda" for the cluster. In particular, it maintains

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanContext.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/PlanContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanContext.java
index 40a25a6..6d3506d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanContext.java
@@ -1,3 +1,21 @@
+/**
+ * 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.Resource;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.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/PlanEdit.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
index 648edba..60e201b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
@@ -1,3 +1,21 @@
+/**
+ * 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.ReservationId;
@@ -34,8 +52,8 @@ public interface PlanEdit extends PlanContext, PlanView {
    * uniquely by its {@link ReservationId}. This will generally be used for
    * garbage collection
    * 
-   * @param reservation the {@link ReservationAllocation} to be deleted from the
-   *          plan identified uniquely by its {@link ReservationId}
+   * @param reservationID the {@link ReservationAllocation} to be deleted from
+   *          the plan identified uniquely by its {@link ReservationId}
    * @return true if delete is successful, false otherwise
    */
   public boolean deleteReservation(ReservationId reservationID)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanFollower.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/PlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanFollower.java
index 9d00366..6635314 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanFollower.java
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 java.util.Collection;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.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/PlanView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
index 6e58dde..b49e99e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 java.util.Set;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/Planner.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/Planner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/Planner.java
new file mode 100644
index 0000000..d2b6184
--- /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/Planner.java
@@ -0,0 +1,48 @@
+/**
+ * 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 java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+
+public interface Planner {
+
+  /**
+   * Update the existing {@link Plan}, by adding/removing/updating existing
+   * reservations, and adding a subset of the reservation requests in the
+   * contracts parameter.
+   *
+   * @param plan the {@link Plan} to replan
+   * @param contracts the list of reservation requests
+   * @throws PlanningException
+   */
+  public void plan(Plan plan, List<ReservationDefinition> contracts)
+      throws PlanningException;
+
+  /**
+   * Initialize the replanner
+   *
+   * @param planQueueName the name of the queue for this plan
+   * @param conf the scheduler configuration
+   */
+  void init(String planQueueName, Configuration conf);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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 fa8db30..3f6f405 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
@@ -1,3 +1,21 @@
+/**
+ * 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 java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAgent.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/ReservationAgent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAgent.java
index fe1941d..6955036 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAgent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAgent.java
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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.ReservationDefinition;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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 bca3aa8..89c0e55 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
@@ -1,3 +1,21 @@
+/**
+ * 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 java.util.Map;
@@ -28,7 +46,7 @@ public interface ReservationAllocation extends
   /**
    * Returns the original {@link ReservationDefinition} submitted by the client
    * 
-   * @return
+   * @return the {@link ReservationDefinition} submitted by the client
    */
   public ReservationDefinition getReservationDefinition();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.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/ReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
index 678773d..fb0831a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
@@ -1,3 +1,21 @@
+/**
+ * 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 java.util.List;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInterval.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/ReservationInterval.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInterval.java
index d3a6d51..aabbcb3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInterval.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInterval.java
@@ -1,3 +1,21 @@
+/**
+ * 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;
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.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/SharingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
index d917764..3e5452e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
@@ -1,3 +1,21 @@
+/**
+ * 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.classification.InterfaceAudience.LimitedPrivate;
@@ -17,10 +35,10 @@ public interface SharingPolicy {
   /**
    * Initialize this policy
    * 
-   * @param inventoryQueuePath the name of the queue for this plan
+   * @param planQueuePath the name of the queue for this plan
    * @param conf the system configuration
    */
-  public void init(String inventoryQueuePath, Configuration conf);
+  public void init(String planQueuePath, Configuration conf);
 
   /**
    * This method runs the policy validation logic, and return true/false on

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SimpleCapacityReplanner.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/SimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SimpleCapacityReplanner.java
index 8384538..5378405 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SimpleCapacityReplanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SimpleCapacityReplanner.java
@@ -1,3 +1,21 @@
+/**
+ * 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 java.util.Iterator;
@@ -7,6 +25,7 @@ import java.util.TreeSet;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
@@ -50,8 +69,10 @@ public class SimpleCapacityReplanner implements Planner {
   }
 
   @Override
-  public void init(String planQueueName, CapacitySchedulerConfiguration conf) {
-    this.lengthOfCheckZone = conf.getEnforcementWindow(planQueueName);
+  public void init(String planQueueName, Configuration conf) {
+    this.lengthOfCheckZone =
+        ((CapacitySchedulerConfiguration) conf)
+            .getEnforcementWindow(planQueueName);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ContractValidationException.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/exceptions/ContractValidationException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ContractValidationException.java
index cd82a9e..c85dbd7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ContractValidationException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ContractValidationException.java
@@ -1,3 +1,21 @@
+/**
+ * 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.exceptions;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/MismatchedUserException.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/exceptions/MismatchedUserException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/MismatchedUserException.java
index 0a443f3..7b4419b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/MismatchedUserException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/MismatchedUserException.java
@@ -1,3 +1,21 @@
+/**
+ * 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.exceptions;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningException.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/exceptions/PlanningException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningException.java
index 0699856..fdfcada 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningException.java
@@ -1,3 +1,21 @@
+/**
+ * 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.exceptions;
 
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;


[11/16] git commit: YARN-2080. Integrating reservation system with ResourceManager and client-RM protocol. Contributed by Subru Krishnan and Carlo Curino. (cherry picked from commit 8baeaead8532898163f1006276b731a237b1a559)

Posted by cd...@apache.org.
YARN-2080. Integrating reservation system with ResourceManager and client-RM protocol. Contributed by Subru Krishnan and Carlo Curino.
(cherry picked from commit 8baeaead8532898163f1006276b731a237b1a559)

Conflicts:
	hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java


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

Branch: refs/heads/trunk
Commit: 6261f7cc69a0eb3eebc9898c7599c7c20f432b4e
Parents: 3418c56
Author: subru <su...@outlook.com>
Authored: Thu Sep 18 15:30:27 2014 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Oct 3 15:42:43 2014 -0700

----------------------------------------------------------------------
 YARN-1051-CHANGES.txt                           |   3 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  19 +
 .../hadoop/yarn/client/api/YarnClient.java      | 102 ++++
 .../yarn/client/api/impl/YarnClientImpl.java    |  25 +
 .../yarn/client/api/impl/TestYarnClient.java    | 115 ++++
 .../server/resourcemanager/AdminService.java    |   6 +
 .../server/resourcemanager/ClientRMService.java | 201 +++++++
 .../server/resourcemanager/RMAuditLogger.java   |   5 +
 .../yarn/server/resourcemanager/RMContext.java  |   2 +
 .../server/resourcemanager/RMContextImpl.java   |  11 +
 .../server/resourcemanager/ResourceManager.java |  38 ++
 .../reservation/AbstractReservationSystem.java  | 323 +++++++++++
 .../reservation/CapacityReservationSystem.java  | 146 +++++
 .../reservation/ReservationInputValidator.java  | 244 ++++++++
 .../reservation/ReservationSystem.java          | 125 +++++
 .../server/resourcemanager/rmapp/RMApp.java     |   3 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |   5 +
 .../scheduler/event/AppAddedSchedulerEvent.java |  16 +-
 .../resourcemanager/TestClientRMService.java    | 116 ++++
 .../TestCapacityReservationSystem.java          | 102 ++++
 .../TestReservationInputValidator.java          | 560 +++++++++++++++++++
 21 files changed, 2165 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/YARN-1051-CHANGES.txt
----------------------------------------------------------------------
diff --git a/YARN-1051-CHANGES.txt b/YARN-1051-CHANGES.txt
index 56b3c12..c4106b2 100644
--- a/YARN-1051-CHANGES.txt
+++ b/YARN-1051-CHANGES.txt
@@ -20,3 +20,6 @@ on user reservations. (Carlo Curino and Subru Krishnan via curino)
 
 YARN-1712. Plan follower that synchronizes the current state of reservation
 subsystem with the scheduler. (Subru Krishnan and Carlo Curino  via subru)
+
+YARN-2080. Integrating reservation system with ResourceManager and 
+client-RM protocol. (Subru Krishnan and Carlo Curino  via subru)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 1a201c4..1a2aa1d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -176,6 +176,25 @@ public class YarnConfiguration extends Configuration {
   public static final boolean DEFAULT_RM_SCHEDULER_USE_PORT_FOR_NODE_NAME = 
       false;
 
+  /** Whether the RM should enable Reservation System */
+  public static final String RM_RESERVATION_SYSTEM_ENABLE = RM_PREFIX
+      + "reservation-system.enable";
+  public static final boolean DEFAULT_RM_RESERVATION_SYSTEM_ENABLE = false;
+
+  /** The class to use as the Reservation System. */
+  public static final String RM_RESERVATION_SYSTEM_CLASS = RM_PREFIX
+      + "reservation-system.class";
+
+  /** The PlanFollower for the Reservation System. */
+  public static final String RM_RESERVATION_SYSTEM_PLAN_FOLLOWER = RM_PREFIX
+      + "reservation-system.plan.follower";
+
+  /** The step size of the Reservation System. */
+  public static final String RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP =
+      RM_PREFIX + "reservation-system.planfollower.time-step";
+  public static final long DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP =
+      1000L;
+
   /**
    * Enable periodic monitor threads.
    * @see #RM_SCHEDULER_MONITOR_POLICIES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
index 9e27de5..d697de9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
@@ -27,10 +27,17 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
@@ -43,6 +50,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
@@ -474,4 +482,98 @@ public abstract class YarnClient extends AbstractService {
    */
   public abstract void moveApplicationAcrossQueues(ApplicationId appId,
       String queue) throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by clients to submit a new reservation to the
+   * {@link ResourceManager}.
+   * </p>
+   * 
+   * <p>
+   * The client packages all details of its request in a
+   * {@link ReservationRequest} object. This contains information about the
+   * amount of capacity, temporal constraints, and gang needs. Furthermore, the
+   * reservation might be composed of multiple stages, with ordering
+   * dependencies among them.
+   * </p>
+   * 
+   * <p>
+   * In order to respond, a new admission control component in the
+   * {@link ResourceManager} performs an analysis of the resources that have
+   * been committed over the period of time the user is requesting, verify that
+   * the user requests can be fulfilled, and that it respect a sharing policy
+   * (e.g., {@link CapacityOverTimePolicy}). Once it has positively determined
+   * that the ReservationRequest is satisfiable the {@link ResourceManager}
+   * answers with a {@link ReservationResponse} that include a
+   * {@link ReservationId}. Upon failure to find a valid allocation the response
+   * is an exception with the message detailing the reason of failure.
+   * </p>
+   * 
+   * <p>
+   * The semantics guarantees that the ReservationId returned, corresponds to a
+   * valid reservation existing in the time-range request by the user. The
+   * amount of capacity dedicated to such reservation can vary overtime,
+   * depending of the allocation that has been determined. But it is guaranteed
+   * to satisfy all the constraint expressed by the user in the
+   * {@link ReservationRequest}
+   * </p>
+   * 
+   * @param request request to submit a new Reservation
+   * @return response contains the {@link ReservationId} on accepting the
+   *         submission
+   * @throws YarnException if the reservation cannot be created successfully
+   * @throws IOException
+   * 
+   */
+  @Public
+  @Unstable
+  public abstract ReservationSubmissionResponse submitReservation(
+      ReservationSubmissionRequest request) throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by clients to update an existing Reservation. This is
+   * referred to as a re-negotiation process, in which a user that has
+   * previously submitted a Reservation.
+   * </p>
+   * 
+   * <p>
+   * The allocation is attempted by virtually substituting all previous
+   * allocations related to this Reservation with new ones, that satisfy the new
+   * {@link ReservationRequest}. Upon success the previous allocation is
+   * atomically substituted by the new one, and on failure (i.e., if the system
+   * cannot find a valid allocation for the updated request), the previous
+   * allocation remains valid.
+   * </p>
+   * 
+   * @param request to update an existing Reservation (the ReservationRequest
+   *          should refer to an existing valid {@link ReservationId})
+   * @return response empty on successfully updating the existing reservation
+   * @throws YarnException if the request is invalid or reservation cannot be
+   *           updated successfully
+   * @throws IOException
+   * 
+   */
+  @Public
+  @Unstable
+  public abstract ReservationUpdateResponse updateReservation(
+      ReservationUpdateRequest request) throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by clients to remove an existing Reservation.
+   * </p>
+   * 
+   * @param request to remove an existing Reservation (the ReservationRequest
+   *          should refer to an existing valid {@link ReservationId})
+   * @return response empty on successfully deleting the existing reservation
+   * @throws YarnException if the request is invalid or reservation cannot be
+   *           deleted successfully
+   * @throws IOException
+   * 
+   */
+  @Public
+  @Unstable
+  public abstract ReservationDeleteResponse deleteReservation(
+      ReservationDeleteRequest request) throws YarnException, IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
index def6da5..02c5a74 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
@@ -63,6 +63,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
@@ -646,4 +652,23 @@ public class YarnClientImpl extends YarnClient {
         MoveApplicationAcrossQueuesRequest.newInstance(appId, queue);
     rmClient.moveApplicationAcrossQueues(request);
   }
+
+  @Override
+  public ReservationSubmissionResponse submitReservation(
+      ReservationSubmissionRequest request) throws YarnException, IOException {
+    return rmClient.submitReservation(request);
+  }
+
+  @Override
+  public ReservationUpdateResponse updateReservation(
+      ReservationUpdateRequest request) throws YarnException, IOException {
+    return rmClient.updateReservation(request);
+  }
+
+  @Override
+  public ReservationDeleteResponse deleteReservation(
+      ReservationDeleteRequest request) throws YarnException, IOException {
+    return rmClient.deleteReservation(request);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
index 3c1b1c1..d7bea7a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
@@ -31,6 +31,7 @@ import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -63,6 +64,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
@@ -76,6 +83,11 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
+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.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
@@ -89,8 +101,14 @@ import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystemTestUtil;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.util.UTCClock;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
@@ -834,4 +852,101 @@ public class TestYarnClient {
       client.stop();
     }
   }
+  
+  @Test
+  public void testReservationAPIs() {
+    // initialize
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    ReservationSystemTestUtil.setupQueueConfiguration(conf);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE, true);
+    MiniYARNCluster cluster =
+        new MiniYARNCluster("testReservationAPIs", 2, 1, 1);
+    YarnClient client = null;
+    try {
+      cluster.init(conf);
+      cluster.start();
+      final Configuration yarnConf = cluster.getConfig();
+      client = YarnClient.createYarnClient();
+      client.init(yarnConf);
+      client.start();
+
+      // create a reservation
+      Clock clock = new UTCClock();
+      long arrival = clock.getTime();
+      long duration = 60000;
+      long deadline = (long) (arrival + 1.05 * duration);
+      ReservationSubmissionRequest sRequest =
+          createSimpleReservationRequest(4, arrival, deadline, duration);
+      ReservationSubmissionResponse sResponse = null;
+      try {
+        sResponse = client.submitReservation(sRequest);
+      } catch (Exception e) {
+        Assert.fail(e.getMessage());
+      }
+      Assert.assertNotNull(sResponse);
+      ReservationId reservationID = sResponse.getReservationId();
+      Assert.assertNotNull(reservationID);
+      System.out.println("Submit reservation response: " + reservationID);
+
+      // Update the reservation
+      ReservationDefinition rDef = sRequest.getReservationDefinition();
+      ReservationRequest rr =
+          rDef.getReservationRequests().getReservationResources().get(0);
+      rr.setNumContainers(5);
+      arrival = clock.getTime();
+      duration = 30000;
+      deadline = (long) (arrival + 1.05 * duration);
+      rr.setDuration(duration);
+      rDef.setArrival(arrival);
+      rDef.setDeadline(deadline);
+      ReservationUpdateRequest uRequest =
+          ReservationUpdateRequest.newInstance(rDef, reservationID);
+      ReservationUpdateResponse uResponse = null;
+      try {
+        uResponse = client.updateReservation(uRequest);
+      } catch (Exception e) {
+        Assert.fail(e.getMessage());
+      }
+      Assert.assertNotNull(sResponse);
+      System.out.println("Update reservation response: " + uResponse);
+
+      // Delete the reservation
+      ReservationDeleteRequest dRequest =
+          ReservationDeleteRequest.newInstance(reservationID);
+      ReservationDeleteResponse dResponse = null;
+      try {
+        dResponse = client.deleteReservation(dRequest);
+      } catch (Exception e) {
+        Assert.fail(e.getMessage());
+      }
+      Assert.assertNotNull(sResponse);
+      System.out.println("Delete reservation response: " + dResponse);
+    } finally {
+      // clean-up
+      if (client != null) {
+        client.stop();
+      }
+      cluster.stop();
+    }
+  }
+
+  private ReservationSubmissionRequest createSimpleReservationRequest(
+      int numContainers, long arrival, long deadline, long duration) {
+    // create a request with a single atomic ask
+    ReservationRequest r =
+        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+            numContainers, 1, duration);
+    ReservationRequests reqs =
+        ReservationRequests.newInstance(Collections.singletonList(r),
+            ReservationRequestInterpreter.R_ALL);
+    ReservationDefinition rDef =
+        ReservationDefinition.newInstance(arrival, deadline, reqs,
+            "testYarnClient#reservation");
+    ReservationSubmissionRequest request =
+        ReservationSubmissionRequest.newInstance(rDef,
+            ReservationSystemTestUtil.reservationQ);
+    return request;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.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/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index ff0a249..2b7797f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMapp
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
@@ -348,6 +349,11 @@ public class AdminService extends CompositeService implements
         recordFactory.newRecordInstance(RefreshQueuesResponse.class);
     try {
       rmContext.getScheduler().reinitialize(getConfig(), this.rmContext);
+      // refresh the reservation system
+      ReservationSystem rSystem = rmContext.getReservationSystem();
+      if (rSystem != null) {
+        rSystem.reinitialize(getConfig(), rmContext);
+      }
       RMAuditLogger.logSuccess(user.getShortUserName(), argName,
           "AdminService");
       return response;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.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/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index 71f873c..12811d0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.net.InetSocketAddress;
 import java.security.AccessControlException;
+import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -79,6 +80,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesReq
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
@@ -93,6 +100,8 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
@@ -107,6 +116,10 @@ import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.Plan;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationInputValidator;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
@@ -123,7 +136,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenS
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.util.UTCClock;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.Futures;
@@ -154,10 +169,23 @@ public class ClientRMService extends AbstractService implements
   private final ApplicationACLsManager applicationsACLsManager;
   private final QueueACLsManager queueACLsManager;
 
+  // For Reservation APIs
+  private Clock clock;
+  private ReservationSystem reservationSystem;
+  private ReservationInputValidator rValidator;
+
   public ClientRMService(RMContext rmContext, YarnScheduler scheduler,
       RMAppManager rmAppManager, ApplicationACLsManager applicationACLsManager,
       QueueACLsManager queueACLsManager,
       RMDelegationTokenSecretManager rmDTSecretManager) {
+    this(rmContext, scheduler, rmAppManager, applicationACLsManager,
+        queueACLsManager, rmDTSecretManager, new UTCClock());
+  }
+
+  public ClientRMService(RMContext rmContext, YarnScheduler scheduler,
+      RMAppManager rmAppManager, ApplicationACLsManager applicationACLsManager,
+      QueueACLsManager queueACLsManager,
+      RMDelegationTokenSecretManager rmDTSecretManager, Clock clock) {
     super(ClientRMService.class.getName());
     this.scheduler = scheduler;
     this.rmContext = rmContext;
@@ -165,6 +193,9 @@ public class ClientRMService extends AbstractService implements
     this.applicationsACLsManager = applicationACLsManager;
     this.queueACLsManager = queueACLsManager;
     this.rmDTSecretManager = rmDTSecretManager;
+    this.reservationSystem = rmContext.getReservationSystem();
+    this.clock = clock;
+    this.rValidator = new ReservationInputValidator(clock);
   }
 
   @Override
@@ -1033,4 +1064,174 @@ public class ClientRMService extends AbstractService implements
   public Server getServer() {
     return this.server;
   }
+
+  @Override
+  public ReservationSubmissionResponse submitReservation(
+      ReservationSubmissionRequest request) throws YarnException, IOException {
+    // Check if reservation system is enabled
+    checkReservationSytem(AuditConstants.SUBMIT_RESERVATION_REQUEST);
+    ReservationSubmissionResponse response =
+        recordFactory.newRecordInstance(ReservationSubmissionResponse.class);
+    // Create a new Reservation Id
+    ReservationId reservationId = reservationSystem.getNewReservationId();
+    // Validate the input
+    Plan plan =
+        rValidator.validateReservationSubmissionRequest(reservationSystem,
+            request, reservationId);
+    // Check ACLs
+    String queueName = request.getQueue();
+    String user =
+        checkReservationACLs(queueName,
+            AuditConstants.SUBMIT_RESERVATION_REQUEST);
+    try {
+      // Try to place the reservation using the agent
+      boolean result =
+          plan.getReservationAgent().createReservation(reservationId, user,
+              plan, request.getReservationDefinition());
+      if (result) {
+        // add the reservation id to valid ones maintained by reservation
+        // system
+        reservationSystem.setQueueForReservation(reservationId, queueName);
+        // create the reservation synchronously if required
+        refreshScheduler(queueName, request.getReservationDefinition(),
+            reservationId.toString());
+        // return the reservation id
+        response.setReservationId(reservationId);
+      }
+    } catch (PlanningException e) {
+      RMAuditLogger.logFailure(user, AuditConstants.SUBMIT_RESERVATION_REQUEST,
+          e.getMessage(), "ClientRMService",
+          "Unable to create the reservation: " + reservationId);
+      throw RPCUtil.getRemoteException(e);
+    }
+    RMAuditLogger.logSuccess(user, AuditConstants.SUBMIT_RESERVATION_REQUEST,
+        "ClientRMService: " + reservationId);
+    return response;
+  }
+
+  @Override
+  public ReservationUpdateResponse updateReservation(
+      ReservationUpdateRequest request) throws YarnException, IOException {
+    // Check if reservation system is enabled
+    checkReservationSytem(AuditConstants.UPDATE_RESERVATION_REQUEST);
+    ReservationUpdateResponse response =
+        recordFactory.newRecordInstance(ReservationUpdateResponse.class);
+    // Validate the input
+    Plan plan =
+        rValidator.validateReservationUpdateRequest(reservationSystem, request);
+    ReservationId reservationId = request.getReservationId();
+    String queueName = reservationSystem.getQueueForReservation(reservationId);
+    // Check ACLs
+    String user =
+        checkReservationACLs(queueName,
+            AuditConstants.UPDATE_RESERVATION_REQUEST);
+    // Try to update the reservation using default agent
+    try {
+      boolean result =
+          plan.getReservationAgent().updateReservation(reservationId, user,
+              plan, request.getReservationDefinition());
+      if (!result) {
+        String errMsg = "Unable to update reservation: " + reservationId;
+        RMAuditLogger.logFailure(user,
+            AuditConstants.UPDATE_RESERVATION_REQUEST, errMsg,
+            "ClientRMService", errMsg);
+        throw RPCUtil.getRemoteException(errMsg);
+      }
+    } catch (PlanningException e) {
+      RMAuditLogger.logFailure(user, AuditConstants.UPDATE_RESERVATION_REQUEST,
+          e.getMessage(), "ClientRMService",
+          "Unable to update the reservation: " + reservationId);
+      throw RPCUtil.getRemoteException(e);
+    }
+    RMAuditLogger.logSuccess(user, AuditConstants.UPDATE_RESERVATION_REQUEST,
+        "ClientRMService: " + reservationId);
+    return response;
+  }
+
+  @Override
+  public ReservationDeleteResponse deleteReservation(
+      ReservationDeleteRequest request) throws YarnException, IOException {
+    // Check if reservation system is enabled
+    checkReservationSytem(AuditConstants.DELETE_RESERVATION_REQUEST);
+    ReservationDeleteResponse response =
+        recordFactory.newRecordInstance(ReservationDeleteResponse.class);
+    // Validate the input
+    Plan plan =
+        rValidator.validateReservationDeleteRequest(reservationSystem, request);
+    ReservationId reservationId = request.getReservationId();
+    String queueName = reservationSystem.getQueueForReservation(reservationId);
+    // Check ACLs
+    String user =
+        checkReservationACLs(queueName,
+            AuditConstants.DELETE_RESERVATION_REQUEST);
+    // Try to update the reservation using default agent
+    try {
+      boolean result =
+          plan.getReservationAgent().deleteReservation(reservationId, user,
+              plan);
+      if (!result) {
+        String errMsg = "Could not delete reservation: " + reservationId;
+        RMAuditLogger.logFailure(user,
+            AuditConstants.DELETE_RESERVATION_REQUEST, errMsg,
+            "ClientRMService", errMsg);
+        throw RPCUtil.getRemoteException(errMsg);
+      }
+    } catch (PlanningException e) {
+      RMAuditLogger.logFailure(user, AuditConstants.DELETE_RESERVATION_REQUEST,
+          e.getMessage(), "ClientRMService",
+          "Unable to delete the reservation: " + reservationId);
+      throw RPCUtil.getRemoteException(e);
+    }
+    RMAuditLogger.logSuccess(user, AuditConstants.DELETE_RESERVATION_REQUEST,
+        "ClientRMService: " + reservationId);
+    return response;
+  }
+
+  private void checkReservationSytem(String auditConstant) throws YarnException {
+    // Check if reservation is enabled
+    if (reservationSystem == null) {
+      throw RPCUtil.getRemoteException("Reservation is not enabled."
+          + " Please enable & try again");
+    }
+  }
+
+  private void refreshScheduler(String planName,
+      ReservationDefinition contract, String reservationId) {
+    if ((contract.getArrival() - clock.getTime()) < reservationSystem
+        .getPlanFollowerTimeStep()) {
+      LOG.debug(MessageFormat
+          .format(
+              "Reservation {0} is within threshold so attempting to create synchronously.",
+              reservationId));
+      reservationSystem.synchronizePlan(planName);
+      LOG.info(MessageFormat.format("Created reservation {0} synchronously.",
+          reservationId));
+    }
+  }
+
+  private String checkReservationACLs(String queueName, String auditConstant)
+      throws YarnException {
+    UserGroupInformation callerUGI;
+    try {
+      callerUGI = UserGroupInformation.getCurrentUser();
+    } catch (IOException ie) {
+      RMAuditLogger.logFailure("UNKNOWN", auditConstant, queueName,
+          "ClientRMService", "Error getting UGI");
+      throw RPCUtil.getRemoteException(ie);
+    }
+    // Check if user has access on the managed queue
+    if (!queueACLsManager.checkAccess(callerUGI, QueueACL.SUBMIT_APPLICATIONS,
+        queueName)) {
+      RMAuditLogger.logFailure(
+          callerUGI.getShortUserName(),
+          auditConstant,
+          "User doesn't have permissions to "
+              + QueueACL.SUBMIT_APPLICATIONS.toString(), "ClientRMService",
+          AuditConstants.UNAUTHORIZED_USER);
+      throw RPCUtil.getRemoteException(new AccessControlException("User "
+          + callerUGI.getShortUserName() + " cannot perform operation "
+          + QueueACL.SUBMIT_APPLICATIONS.name() + " on queue" + queueName));
+    }
+    return callerUGI.getShortUserName();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.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/RMAuditLogger.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java
index 9ae09a4..6dd67c7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java
@@ -56,6 +56,11 @@ public class RMAuditLogger {
 
     // Some commonly used descriptions
     public static final String UNAUTHORIZED_USER = "Unauthorized user";
+    
+    // For Reservation system
+    public static final String SUBMIT_RESERVATION_REQUEST = "Submit Reservation Request";
+    public static final String UPDATE_RESERVATION_REQUEST = "Update Reservation Request";
+    public static final String DELETE_RESERVATION_REQUEST = "Delete Reservation Request";
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.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/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index 60f88f6..46ecfcd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -109,5 +109,7 @@ public interface RMContext {
   
   long getEpoch();
 
+  ReservationSystem getReservationSystem();
+
   boolean isSchedulerReadyForAllocatingContainers();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.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/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index 36eec04..78787ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWri
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
@@ -83,6 +84,7 @@ public class RMContextImpl implements RMContext {
   private ClientRMService clientRMService;
   private RMDelegationTokenSecretManager rmDelegationTokenSecretManager;
   private ResourceScheduler scheduler;
+  private ReservationSystem reservationSystem;
   private NodesListManager nodesListManager;
   private ResourceTrackerService resourceTrackerService;
   private ApplicationMasterService applicationMasterService;
@@ -209,6 +211,11 @@ public class RMContextImpl implements RMContext {
   }
 
   @Override
+  public ReservationSystem getReservationSystem() {
+    return this.reservationSystem;
+  }
+  
+  @Override
   public NodesListManager getNodesListManager() {
     return this.nodesListManager;
   }
@@ -303,6 +310,10 @@ public class RMContextImpl implements RMContext {
   void setScheduler(ResourceScheduler scheduler) {
     this.scheduler = scheduler;
   }
+  
+  void setReservationSystem(ReservationSystem reservationSystem) {
+    this.reservationSystem = reservationSystem;
+  }
 
   void setDelegationTokenRenewer(
       DelegationTokenRenewer delegationTokenRenewer) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.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/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 79af7a6..3e5f138 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -72,6 +72,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreFactory;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.AbstractReservationSystem;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
@@ -147,6 +149,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
   protected RMSecretManagerService rmSecretManagerService;
 
   protected ResourceScheduler scheduler;
+  protected ReservationSystem reservationSystem;
   private ClientRMService clientRM;
   protected ApplicationMasterService masterService;
   protected NMLivelinessMonitor nmLivelinessMonitor;
@@ -281,6 +284,29 @@ public class ResourceManager extends CompositeService implements Recoverable {
     }
   }
 
+  protected ReservationSystem createReservationSystem() {
+    String reservationClassName =
+        conf.get(YarnConfiguration.RM_RESERVATION_SYSTEM_CLASS,
+            AbstractReservationSystem.getDefaultReservationSystem(scheduler));
+    if (reservationClassName == null) {
+      return null;
+    }
+    LOG.info("Using ReservationSystem: " + reservationClassName);
+    try {
+      Class<?> reservationClazz = Class.forName(reservationClassName);
+      if (ReservationSystem.class.isAssignableFrom(reservationClazz)) {
+        return (ReservationSystem) ReflectionUtils.newInstance(
+            reservationClazz, this.conf);
+      } else {
+        throw new YarnRuntimeException("Class: " + reservationClassName
+            + " not instance of " + ReservationSystem.class.getCanonicalName());
+      }
+    } catch (ClassNotFoundException e) {
+      throw new YarnRuntimeException(
+          "Could not instantiate ReservationSystem: " + reservationClassName, e);
+    }
+  }
+
   protected ApplicationMasterLauncher createAMLauncher() {
     return new ApplicationMasterLauncher(this.rmContext);
   }
@@ -456,6 +482,18 @@ public class ResourceManager extends CompositeService implements Recoverable {
       DefaultMetricsSystem.initialize("ResourceManager");
       JvmMetrics.initSingleton("ResourceManager", null);
 
+      // Initialize the Reservation system
+      if (conf.getBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE,
+          YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_ENABLE)) {
+        reservationSystem = createReservationSystem();
+        if (reservationSystem != null) {
+          reservationSystem.setRMContext(rmContext);
+          addIfService(reservationSystem);
+          rmContext.setReservationSystem(reservationSystem);
+          LOG.info("Initialized Reservation system");
+        }
+      }
+
       // creating monitors that handle preemption
       createPolicyMonitors();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.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/AbstractReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
new file mode 100644
index 0000000..f0a9543
--- /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/AbstractReservationSystem.java
@@ -0,0 +1,323 @@
+/**
+ *
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.UTCClock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is the implementation of {@link ReservationSystem} based on the
+ * {@link ResourceScheduler}
+ */
+@LimitedPrivate("yarn")
+@Unstable
+public abstract class AbstractReservationSystem extends AbstractService
+    implements ReservationSystem {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AbstractReservationSystem.class);
+
+  // private static final String DEFAULT_CAPACITY_SCHEDULER_PLAN
+
+  private final ReentrantReadWriteLock readWriteLock =
+      new ReentrantReadWriteLock(true);
+  private final Lock readLock = readWriteLock.readLock();
+  private final Lock writeLock = readWriteLock.writeLock();
+
+  private boolean initialized = false;
+
+  private final Clock clock = new UTCClock();
+
+  private AtomicLong resCounter = new AtomicLong();
+
+  private Map<String, Plan> plans = new HashMap<String, Plan>();
+
+  private Map<ReservationId, String> resQMap =
+      new HashMap<ReservationId, String>();
+
+  private RMContext rmContext;
+
+  private ResourceScheduler scheduler;
+
+  private ScheduledExecutorService scheduledExecutorService;
+
+  protected Configuration conf;
+
+  protected long planStepSize;
+
+  private PlanFollower planFollower;
+
+  /**
+   * Construct the service.
+   * 
+   * @param name service name
+   */
+  public AbstractReservationSystem(String name) {
+    super(name);
+  }
+
+  @Override
+  public void setRMContext(RMContext rmContext) {
+    writeLock.lock();
+    try {
+      this.rmContext = rmContext;
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  @Override
+  public void reinitialize(Configuration conf, RMContext rmContext)
+      throws YarnException {
+    writeLock.lock();
+    try {
+      if (!initialized) {
+        initialize(conf);
+        initialized = true;
+      } else {
+        initializeNewPlans(conf);
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private void initialize(Configuration conf) throws YarnException {
+    LOG.info("Initializing Reservation system");
+    this.conf = conf;
+    scheduler = rmContext.getScheduler();
+    // Get the plan step size
+    planStepSize =
+        conf.getTimeDuration(
+            YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+            YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP,
+            TimeUnit.MILLISECONDS);
+    if (planStepSize < 0) {
+      planStepSize =
+          YarnConfiguration.DEFAULT_RM_RESERVATION_SYSTEM_PLAN_FOLLOWER_TIME_STEP;
+    }
+    // Create a plan corresponding to every reservable queue
+    Set<String> planQueueNames = scheduler.getPlanQueues();
+    for (String planQueueName : planQueueNames) {
+      Plan plan = initializePlan(planQueueName);
+      plans.put(planQueueName, plan);
+    }
+  }
+
+  private void initializeNewPlans(Configuration conf) {
+    LOG.info("Refreshing Reservation system");
+    writeLock.lock();
+    try {
+      // Create a plan corresponding to every new reservable queue
+      Set<String> planQueueNames = scheduler.getPlanQueues();
+      for (String planQueueName : planQueueNames) {
+        if (!plans.containsKey(planQueueName)) {
+          Plan plan = initializePlan(planQueueName);
+          plans.put(planQueueName, plan);
+        } else {
+          LOG.warn("Plan based on reservation queue {0} already exists.",
+              planQueueName);
+        }
+      }
+      // Update the plan follower with the active plans
+      if (planFollower != null) {
+        planFollower.setPlans(plans.values());
+      }
+    } catch (YarnException e) {
+      LOG.warn("Exception while trying to refresh reservable queues", e);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private PlanFollower createPlanFollower() {
+    String planFollowerPolicyClassName =
+        conf.get(YarnConfiguration.RM_RESERVATION_SYSTEM_PLAN_FOLLOWER,
+            getDefaultPlanFollower());
+    if (planFollowerPolicyClassName == null) {
+      return null;
+    }
+    LOG.info("Using PlanFollowerPolicy: " + planFollowerPolicyClassName);
+    try {
+      Class<?> planFollowerPolicyClazz =
+          conf.getClassByName(planFollowerPolicyClassName);
+      if (PlanFollower.class.isAssignableFrom(planFollowerPolicyClazz)) {
+        return (PlanFollower) ReflectionUtils.newInstance(
+            planFollowerPolicyClazz, conf);
+      } else {
+        throw new YarnRuntimeException("Class: " + planFollowerPolicyClassName
+            + " not instance of " + PlanFollower.class.getCanonicalName());
+      }
+    } catch (ClassNotFoundException e) {
+      throw new YarnRuntimeException(
+          "Could not instantiate PlanFollowerPolicy: "
+              + planFollowerPolicyClassName, e);
+    }
+  }
+
+  private String getDefaultPlanFollower() {
+    // currently only capacity scheduler is supported
+    if (scheduler instanceof CapacityScheduler) {
+      return CapacitySchedulerPlanFollower.class.getName();
+    }
+    return null;
+  }
+
+  @Override
+  public Plan getPlan(String planName) {
+    readLock.lock();
+    try {
+      return plans.get(planName);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * @return the planStepSize
+   */
+  @Override
+  public long getPlanFollowerTimeStep() {
+    readLock.lock();
+    try {
+      return planStepSize;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public void synchronizePlan(String planName) {
+    writeLock.lock();
+    try {
+      Plan plan = plans.get(planName);
+      if (plan != null) {
+        planFollower.synchronizePlan(plan);
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    Configuration configuration = new Configuration(conf);
+    reinitialize(configuration, rmContext);
+    // Create the plan follower with the active plans
+    planFollower = createPlanFollower();
+    if (planFollower != null) {
+      planFollower.init(clock, scheduler, plans.values());
+    }
+    super.serviceInit(conf);
+  }
+
+  @Override
+  public void serviceStart() throws Exception {
+    if (planFollower != null) {
+      scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
+      scheduledExecutorService.scheduleWithFixedDelay(planFollower, 0L,
+          planStepSize, TimeUnit.MILLISECONDS);
+    }
+    super.serviceStart();
+  }
+
+  @Override
+  public void serviceStop() {
+    // Stop the plan follower
+    if (scheduledExecutorService != null
+        && !scheduledExecutorService.isShutdown()) {
+      scheduledExecutorService.shutdown();
+    }
+    // Clear the plans
+    plans.clear();
+  }
+
+  @Override
+  public String getQueueForReservation(ReservationId reservationId) {
+    readLock.lock();
+    try {
+      return resQMap.get(reservationId);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public void setQueueForReservation(ReservationId reservationId,
+      String queueName) {
+    writeLock.lock();
+    try {
+      resQMap.put(reservationId, queueName);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  @Override
+  public ReservationId getNewReservationId() {
+    writeLock.lock();
+    try {
+      ReservationId resId =
+          ReservationId.newInstance(ResourceManager.getClusterTimeStamp(),
+              resCounter.incrementAndGet());
+      LOG.info("Allocated new reservationId: " + resId);
+      return resId;
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  @Override
+  public Map<String, Plan> getAllPlans() {
+    return plans;
+  }
+
+  /**
+   * Get the default reservation system corresponding to the scheduler
+   * 
+   * @param scheduler the scheduler for which the reservation system is required
+   */
+  public static String getDefaultReservationSystem(ResourceScheduler scheduler) {
+    // currently only capacity scheduler is supported
+    if (scheduler instanceof CapacityScheduler) {
+      return CapacityReservationSystem.class.getName();
+    }
+    return null;
+  }
+
+  protected abstract Plan initializePlan(String planQueueName)
+      throws YarnException;
+
+  protected abstract Planner getReplanner(String planQueueName);
+
+  protected abstract ReservationAgent getAgent(String queueName);
+
+  protected abstract SharingPolicy getAdmissionPolicy(String queueName);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityReservationSystem.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/CapacityReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityReservationSystem.java
new file mode 100644
index 0000000..548fde1
--- /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/CapacityReservationSystem.java
@@ -0,0 +1,146 @@
+/**
+ *
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is the implementation of {@link ReservationSystem} based on the
+ * {@link CapacityScheduler}
+ */
+@LimitedPrivate("yarn")
+@Unstable
+public class CapacityReservationSystem extends AbstractReservationSystem {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(CapacityReservationSystem.class);
+
+  private CapacityScheduler capScheduler;
+
+  public CapacityReservationSystem() {
+    super(CapacityReservationSystem.class.getName());
+  }
+
+  @Override
+  public void reinitialize(Configuration conf, RMContext rmContext)
+      throws YarnException {
+    // Validate if the scheduler is capacity based
+    ResourceScheduler scheduler = rmContext.getScheduler();
+    if (!(scheduler instanceof CapacityScheduler)) {
+      throw new YarnRuntimeException("Class "
+          + scheduler.getClass().getCanonicalName() + " not instance of "
+          + CapacityScheduler.class.getCanonicalName());
+    }
+    capScheduler = (CapacityScheduler) scheduler;
+    this.conf = conf;
+    super.reinitialize(conf, rmContext);
+  }
+
+  @Override
+  protected Plan initializePlan(String planQueueName) throws YarnException {
+    SharingPolicy adPolicy = getAdmissionPolicy(planQueueName);
+    String planQueuePath = capScheduler.getQueue(planQueueName).getQueuePath();
+    adPolicy.init(planQueuePath, capScheduler.getConfiguration());
+    CSQueue planQueue = capScheduler.getQueue(planQueueName);
+    // Calculate the max plan capacity
+    Resource minAllocation = capScheduler.getMinimumResourceCapability();
+    ResourceCalculator rescCalc = capScheduler.getResourceCalculator();
+    Resource totCap =
+        rescCalc.multiplyAndNormalizeDown(capScheduler.getClusterResource(),
+            planQueue.getAbsoluteCapacity(), minAllocation);
+    Plan plan =
+        new InMemoryPlan(capScheduler.getRootQueueMetrics(), adPolicy,
+            getAgent(planQueuePath), totCap, planStepSize, rescCalc,
+            minAllocation, capScheduler.getMaximumResourceCapability(),
+            planQueueName, getReplanner(planQueuePath), capScheduler
+                .getConfiguration().getMoveOnExpiry(planQueuePath));
+    LOG.info("Intialized plan {0} based on reservable queue {1}",
+        plan.toString(), planQueueName);
+    return plan;
+  }
+
+  @Override
+  protected Planner getReplanner(String planQueueName) {
+    CapacitySchedulerConfiguration capSchedulerConfig =
+        capScheduler.getConfiguration();
+    String plannerClassName = capSchedulerConfig.getReplanner(planQueueName);
+    LOG.info("Using Replanner: " + plannerClassName + " for queue: "
+        + planQueueName);
+    try {
+      Class<?> plannerClazz =
+          capSchedulerConfig.getClassByName(plannerClassName);
+      if (Planner.class.isAssignableFrom(plannerClazz)) {
+        Planner planner =
+            (Planner) ReflectionUtils.newInstance(plannerClazz, conf);
+        planner.init(planQueueName, capSchedulerConfig);
+        return planner;
+      } else {
+        throw new YarnRuntimeException("Class: " + plannerClazz
+            + " not instance of " + Planner.class.getCanonicalName());
+      }
+    } catch (ClassNotFoundException e) {
+      throw new YarnRuntimeException("Could not instantiate Planner: "
+          + plannerClassName + " for queue: " + planQueueName, e);
+    }
+  }
+
+  @Override
+  protected ReservationAgent getAgent(String queueName) {
+    CapacitySchedulerConfiguration capSchedulerConfig =
+        capScheduler.getConfiguration();
+    String agentClassName = capSchedulerConfig.getReservationAgent(queueName);
+    LOG.info("Using Agent: " + agentClassName + " for queue: " + queueName);
+    try {
+      Class<?> agentClazz = capSchedulerConfig.getClassByName(agentClassName);
+      if (ReservationAgent.class.isAssignableFrom(agentClazz)) {
+        return (ReservationAgent) ReflectionUtils.newInstance(agentClazz, conf);
+      } else {
+        throw new YarnRuntimeException("Class: " + agentClassName
+            + " not instance of " + ReservationAgent.class.getCanonicalName());
+      }
+    } catch (ClassNotFoundException e) {
+      throw new YarnRuntimeException("Could not instantiate Agent: "
+          + agentClassName + " for queue: " + queueName, e);
+    }
+  }
+
+  @Override
+  protected SharingPolicy getAdmissionPolicy(String queueName) {
+    CapacitySchedulerConfiguration capSchedulerConfig =
+        capScheduler.getConfiguration();
+    String admissionPolicyClassName =
+        capSchedulerConfig.getReservationAdmissionPolicy(queueName);
+    LOG.info("Using AdmissionPolicy: " + admissionPolicyClassName
+        + " for queue: " + queueName);
+    try {
+      Class<?> admissionPolicyClazz =
+          capSchedulerConfig.getClassByName(admissionPolicyClassName);
+      if (SharingPolicy.class.isAssignableFrom(admissionPolicyClazz)) {
+        return (SharingPolicy) ReflectionUtils.newInstance(
+            admissionPolicyClazz, conf);
+      } else {
+        throw new YarnRuntimeException("Class: " + admissionPolicyClassName
+            + " not instance of " + SharingPolicy.class.getCanonicalName());
+      }
+    } catch (ClassNotFoundException e) {
+      throw new YarnRuntimeException("Could not instantiate AdmissionPolicy: "
+          + admissionPolicyClassName + " for queue: " + queueName, e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.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/ReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
new file mode 100644
index 0000000..678773d
--- /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/ReservationInputValidator.java
@@ -0,0 +1,244 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+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.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+public class ReservationInputValidator {
+
+  private final Clock clock;
+
+  /**
+   * Utility class to validate reservation requests.
+   */
+  public ReservationInputValidator(Clock clock) {
+    this.clock = clock;
+  }
+
+  private Plan validateReservation(ReservationSystem reservationSystem,
+      ReservationId reservationId, String auditConstant) throws YarnException {
+    String message = "";
+    // check if the reservation id is valid
+    if (reservationId == null) {
+      message =
+          "Missing reservation id."
+              + " Please try again by specifying a reservation id.";
+      RMAuditLogger.logFailure("UNKNOWN", auditConstant,
+          "validate reservation input", "ClientRMService", message);
+      throw RPCUtil.getRemoteException(message);
+    }
+    String queueName = reservationSystem.getQueueForReservation(reservationId);
+    if (queueName == null) {
+      message =
+          "The specified reservation with ID: " + reservationId
+              + " is unknown. Please try again with a valid reservation.";
+      RMAuditLogger.logFailure("UNKNOWN", auditConstant,
+          "validate reservation input", "ClientRMService", message);
+      throw RPCUtil.getRemoteException(message);
+    }
+    // check if the associated plan is valid
+    Plan plan = reservationSystem.getPlan(queueName);
+    if (plan == null) {
+      message =
+          "The specified reservation: " + reservationId
+              + " is not associated with any valid plan."
+              + " Please try again with a valid reservation.";
+      RMAuditLogger.logFailure("UNKNOWN", auditConstant,
+          "validate reservation input", "ClientRMService", message);
+      throw RPCUtil.getRemoteException(message);
+    }
+    return plan;
+  }
+
+  private void validateReservationDefinition(ReservationId reservationId,
+      ReservationDefinition contract, Plan plan, String auditConstant)
+      throws YarnException {
+    String message = "";
+    // check if deadline is in the past
+    if (contract == null) {
+      message =
+          "Missing reservation definition."
+              + " Please try again by specifying a reservation definition.";
+      RMAuditLogger.logFailure("UNKNOWN", auditConstant,
+          "validate reservation input definition", "ClientRMService", message);
+      throw RPCUtil.getRemoteException(message);
+    }
+    if (contract.getDeadline() <= clock.getTime()) {
+      message =
+          "The specified deadline: " + contract.getDeadline()
+              + " is the past. Please try again with deadline in the future.";
+      RMAuditLogger.logFailure("UNKNOWN", auditConstant,
+          "validate reservation input definition", "ClientRMService", message);
+      throw RPCUtil.getRemoteException(message);
+    }
+    // Check if at least one RR has been specified
+    ReservationRequests resReqs = contract.getReservationRequests();
+    if (resReqs == null) {
+      message =
+          "No resources have been specified to reserve."
+              + "Please try again by specifying the resources to reserve.";
+      RMAuditLogger.logFailure("UNKNOWN", auditConstant,
+          "validate reservation input definition", "ClientRMService", message);
+      throw RPCUtil.getRemoteException(message);
+    }
+    List<ReservationRequest> resReq = resReqs.getReservationResources();
+    if (resReq == null || resReq.isEmpty()) {
+      message =
+          "No resources have been specified to reserve."
+              + " Please try again by specifying the resources to reserve.";
+      RMAuditLogger.logFailure("UNKNOWN", auditConstant,
+          "validate reservation input definition", "ClientRMService", message);
+      throw RPCUtil.getRemoteException(message);
+    }
+    // compute minimum duration and max gang size
+    long minDuration = 0;
+    Resource maxGangSize = Resource.newInstance(0, 0);
+    ReservationRequestInterpreter type =
+        contract.getReservationRequests().getInterpreter();
+    for (ReservationRequest rr : resReq) {
+      if (type == ReservationRequestInterpreter.R_ALL
+          || type == ReservationRequestInterpreter.R_ANY) {
+        minDuration = Math.max(minDuration, rr.getDuration());
+      } else {
+        minDuration += rr.getDuration();
+      }
+      maxGangSize =
+          Resources.max(plan.getResourceCalculator(), plan.getTotalCapacity(),
+              maxGangSize,
+              Resources.multiply(rr.getCapability(), rr.getConcurrency()));
+    }
+    // verify the allocation is possible (skip for ANY)
+    if (contract.getDeadline() - contract.getArrival() < minDuration
+        && type != ReservationRequestInterpreter.R_ANY) {
+      message =
+          "The time difference ("
+              + (contract.getDeadline() - contract.getArrival())
+              + ") between arrival (" + contract.getArrival() + ") "
+              + "and deadline (" + contract.getDeadline() + ") must "
+              + " be greater or equal to the minimum resource duration ("
+              + minDuration + ")";
+      RMAuditLogger.logFailure("UNKNOWN", auditConstant,
+          "validate reservation input definition", "ClientRMService", message);
+      throw RPCUtil.getRemoteException(message);
+    }
+    // check that the largest gang does not exceed the inventory available
+    // capacity (skip for ANY)
+    if (Resources.greaterThan(plan.getResourceCalculator(),
+        plan.getTotalCapacity(), maxGangSize, plan.getTotalCapacity())
+        && type != ReservationRequestInterpreter.R_ANY) {
+      message =
+          "The size of the largest gang in the reservation refinition ("
+              + maxGangSize + ") exceed the capacity available ("
+              + plan.getTotalCapacity() + " )";
+      RMAuditLogger.logFailure("UNKNOWN", auditConstant,
+          "validate reservation input definition", "ClientRMService", message);
+      throw RPCUtil.getRemoteException(message);
+    }
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast) the input and returns the appropriate {@link Plan} associated with
+   * the specified {@link Queue} or throws an exception message illustrating the
+   * details of any validation check failures
+   * 
+   * @param reservationSystem the {@link ReservationSystem} to validate against
+   * @param request the {@link ReservationSubmissionRequest} defining the
+   *          resources required over time for the request
+   * @param reservationId the {@link ReservationId} associated with the current
+   *          request
+   * @return the {@link Plan} to submit the request to
+   * @throws YarnException
+   */
+  public Plan validateReservationSubmissionRequest(
+      ReservationSystem reservationSystem,
+      ReservationSubmissionRequest request, ReservationId reservationId)
+      throws YarnException {
+    // Check if it is a managed queue
+    String queueName = request.getQueue();
+    if (queueName == null || queueName.isEmpty()) {
+      String errMsg =
+          "The queue to submit is not specified."
+              + " Please try again with a valid reservable queue.";
+      RMAuditLogger.logFailure("UNKNOWN",
+          AuditConstants.SUBMIT_RESERVATION_REQUEST,
+          "validate reservation input", "ClientRMService", errMsg);
+      throw RPCUtil.getRemoteException(errMsg);
+    }
+    Plan plan = reservationSystem.getPlan(queueName);
+    if (plan == null) {
+      String errMsg =
+          "The specified queue: " + queueName
+              + " is not managed by reservation system."
+              + " Please try again with a valid reservable queue.";
+      RMAuditLogger.logFailure("UNKNOWN",
+          AuditConstants.SUBMIT_RESERVATION_REQUEST,
+          "validate reservation input", "ClientRMService", errMsg);
+      throw RPCUtil.getRemoteException(errMsg);
+    }
+    validateReservationDefinition(reservationId,
+        request.getReservationDefinition(), plan,
+        AuditConstants.SUBMIT_RESERVATION_REQUEST);
+    return plan;
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast) the input and returns the appropriate {@link Plan} associated with
+   * the specified {@link Queue} or throws an exception message illustrating the
+   * details of any validation check failures
+   * 
+   * @param reservationSystem the {@link ReservationSystem} to validate against
+   * @param request the {@link ReservationUpdateRequest} defining the resources
+   *          required over time for the request
+   * @return the {@link Plan} to submit the request to
+   * @throws YarnException
+   */
+  public Plan validateReservationUpdateRequest(
+      ReservationSystem reservationSystem, ReservationUpdateRequest request)
+      throws YarnException {
+    ReservationId reservationId = request.getReservationId();
+    Plan plan =
+        validateReservation(reservationSystem, reservationId,
+            AuditConstants.UPDATE_RESERVATION_REQUEST);
+    validateReservationDefinition(reservationId,
+        request.getReservationDefinition(), plan,
+        AuditConstants.UPDATE_RESERVATION_REQUEST);
+    return plan;
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast) the input and returns the appropriate {@link Plan} associated with
+   * the specified {@link Queue} or throws an exception message illustrating the
+   * details of any validation check failures
+   * 
+   * @param reservationSystem the {@link ReservationSystem} to validate against
+   * @param request the {@link ReservationDeleteRequest} defining the resources
+   *          required over time for the request
+   * @return the {@link Plan} to submit the request to
+   * @throws YarnException
+   */
+  public Plan validateReservationDeleteRequest(
+      ReservationSystem reservationSystem, ReservationDeleteRequest request)
+      throws YarnException {
+    return validateReservation(reservationSystem, request.getReservationId(),
+        AuditConstants.DELETE_RESERVATION_REQUEST);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.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/ReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystem.java
new file mode 100644
index 0000000..cb76dcf
--- /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/ReservationSystem.java
@@ -0,0 +1,125 @@
+/**
+ * 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 java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+
+/**
+ * This interface is the one implemented by any system that wants to support
+ * Reservations i.e. make {@link Resource} allocations in future. Implementors
+ * need to bootstrap all configured {@link Plan}s in the active
+ * {@link ResourceScheduler} along with their corresponding
+ * {@link ReservationAgent} and {@link SharingPolicy}. It is also responsible
+ * for managing the {@link PlanFollower} to ensure the {@link Plan}s are in sync
+ * with the {@link ResourceScheduler}.
+ */
+@LimitedPrivate("yarn")
+@Unstable
+public interface ReservationSystem {
+
+  /**
+   * Set RMContext for {@link ReservationSystem}. This method should be called
+   * immediately after instantiating a reservation system once.
+   * 
+   * @param rmContext created by {@link ResourceManager}
+   */
+  void setRMContext(RMContext rmContext);
+
+  /**
+   * Re-initialize the {@link ReservationSystem}.
+   * 
+   * @param conf configuration
+   * @param rmContext current context of the {@link ResourceManager}
+   * @throws YarnException
+   */
+  void reinitialize(Configuration conf, RMContext rmContext)
+      throws YarnException;
+
+  /**
+   * Get an existing {@link Plan} that has been initialized.
+   * 
+   * @param planName the name of the {@link Plan}
+   * @return the {@link Plan} identified by name
+   * 
+   */
+  Plan getPlan(String planName);
+
+  /**
+   * Return a map containing all the plans known to this ReservationSystem
+   * (useful for UI)
+   * 
+   * @return a Map of Plan names and Plan objects
+   */
+  Map<String, Plan> getAllPlans();
+
+  /**
+   * Invokes {@link PlanFollower} to synchronize the specified {@link Plan} with
+   * the {@link ResourceScheduler}
+   * 
+   * @param planName the name of the {@link Plan} to be synchronized
+   */
+  void synchronizePlan(String planName);
+
+  /**
+   * Return the time step (ms) at which the {@link PlanFollower} is invoked
+   * 
+   * @return the time step (ms) at which the {@link PlanFollower} is invoked
+   */
+  long getPlanFollowerTimeStep();
+
+  /**
+   * Get a new unique {@link ReservationId}.
+   * 
+   * @return a new unique {@link ReservationId}
+   * 
+   */
+  ReservationId getNewReservationId();
+
+  /**
+   * Get the {@link Queue} that an existing {@link ReservationId} is associated
+   * with.
+   * 
+   * @param reservationId the unique id of the reservation
+   * @return the name of the associated Queue
+   * 
+   */
+  String getQueueForReservation(ReservationId reservationId);
+
+  /**
+   * Set the {@link Queue} that an existing {@link ReservationId} should be
+   * associated with.
+   * 
+   * @param reservationId the unique id of the reservation
+   * @param queueName the name of Queue to associate the reservation with
+   * 
+   */
+  void setQueueForReservation(ReservationId reservationId, String queueName);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.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/rmapp/RMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
index a1ae3ca..624aa18 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -236,4 +237,6 @@ public interface RMApp extends EventHandler<RMAppEvent> {
    * @return metrics
    */
   RMAppMetrics getRMAppMetrics();
+
+  ReservationId getReservationId();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.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/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 4899434..84ec766 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -1284,4 +1285,8 @@ public class RMAppImpl implements RMApp, Recoverable {
     this.systemClock = clock;
   }
 
+  @Override
+  public ReservationId getReservationId() {
+    return submissionContext.getReservationID();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6261f7cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAddedSchedulerEvent.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/scheduler/event/AppAddedSchedulerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAddedSchedulerEvent.java
index 7e0b89e..a54e4bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAddedSchedulerEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAddedSchedulerEvent.java
@@ -19,25 +19,33 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.event;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 
 public class AppAddedSchedulerEvent extends SchedulerEvent {
 
   private final ApplicationId applicationId;
   private final String queue;
   private final String user;
+  private final ReservationId reservationID;
   private final boolean isAppRecovering;
 
   public AppAddedSchedulerEvent(
       ApplicationId applicationId, String queue, String user) {
-    this(applicationId, queue, user, false);
+    this(applicationId, queue, user, false, null);
   }
 
   public AppAddedSchedulerEvent(ApplicationId applicationId, String queue,
-      String user, boolean isAppRecovering) {
+      String user, ReservationId reservationID) {
+    this(applicationId, queue, user, false, reservationID);
+  }
+
+  public AppAddedSchedulerEvent(ApplicationId applicationId, String queue,
+      String user, boolean isAppRecovering, ReservationId reservationID) {
     super(SchedulerEventType.APP_ADDED);
     this.applicationId = applicationId;
     this.queue = queue;
     this.user = user;
+    this.reservationID = reservationID;
     this.isAppRecovering = isAppRecovering;
   }
 
@@ -56,4 +64,8 @@ public class AppAddedSchedulerEvent extends SchedulerEvent {
   public boolean getIsAppRecovering() {
     return isAppRecovering;
   }
+
+  public ReservationId getReservationID() {
+    return reservationID;
+  }
 }


[13/16] YARN-2576. Making test patch pass in branch. Contributed by Subru Krishnan and Carlo Curino. (cherry picked from commit 90ac0be86b898aefec5471db4027554c8e1b310c)

Posted by cd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningQuotaException.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/exceptions/PlanningQuotaException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningQuotaException.java
index aad4ee8..5c0ffc4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningQuotaException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningQuotaException.java
@@ -1,3 +1,21 @@
+/**
+ * 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.exceptions;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ResourceOverCommitException.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/exceptions/ResourceOverCommitException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ResourceOverCommitException.java
index a4c2b07..87f8837 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ResourceOverCommitException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ResourceOverCommitException.java
@@ -1,7 +1,26 @@
+/**
+ * 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.exceptions;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.Plan;
 
 /**
  * This exception indicate that the reservation that has been attempted, would

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerDynamicEditException.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/scheduler/SchedulerDynamicEditException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerDynamicEditException.java
index 42dc36a..606332c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerDynamicEditException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerDynamicEditException.java
@@ -1,3 +1,21 @@
+/**
+ * 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;
 
 import org.apache.hadoop.yarn.exceptions.YarnException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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
----------------------------------------------------------------------
diff --git a/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 b/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
index ff6db3a..6a3c7dc 100644
--- a/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
+++ b/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
@@ -522,7 +522,7 @@ public class CapacityScheduler extends
             : (parent.getQueuePath() + "." + queueName);
     String[] childQueueNames = 
       conf.getQueues(fullQueueName);
-    boolean isReservableQueue = conf.isReservableQueue(fullQueueName);
+    boolean isReservableQueue = conf.isReservable(fullQueueName);
     if (childQueueNames == null || childQueueNames.length == 0) {
       if (null == parent) {
         throw new IllegalStateException(
@@ -1284,6 +1284,15 @@ public class CapacityScheduler extends
             .handle(new RMAppRejectedEvent(applicationId, message));
         return null;
       }
+      if (!queue.getParent().getQueueName().equals(queueName)) {
+        String message =
+            "Application: " + applicationId + " submitted to a reservation "
+                + resQName + " which does not belong to the specified queue: "
+                + queueName;
+        this.rmContext.getDispatcher().getEventHandler()
+            .handle(new RMAppRejectedEvent(applicationId, message));
+        return null;
+      }
       // use the reservation queue to run the app
       queueName = resQName;
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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
----------------------------------------------------------------------
diff --git a/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 b/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
index ba501b6..b1f239c 100644
--- a/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
+++ b/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
@@ -204,7 +204,7 @@ public class CapacitySchedulerConfiguration extends Configuration {
       "instantaneous-max-capacity";
 
   @Private
-  public static final long DEFAULT_RESERVATION_WINDOW = 0L;
+  public static final long DEFAULT_RESERVATION_WINDOW = 86400000L;
 
   @Private
   public static final String RESERVATION_ADMISSION_POLICY =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java
index 4ada778..abb8e95 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java
@@ -1,3 +1,21 @@
+/**
+ * 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.capacity;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java
index 48733fc..8e61821 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java
@@ -1,3 +1,21 @@
+/**
+ * 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.capacity;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/QueueEntitlement.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/scheduler/common/QueueEntitlement.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/QueueEntitlement.java
index a348e13..2a751e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/QueueEntitlement.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/QueueEntitlement.java
@@ -1,3 +1,21 @@
+/**
+ * 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.common;
 
 public class QueueEntitlement {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.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/webapp/dao/CapacitySchedulerQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
index 0a8faad..d5c84d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
@@ -49,7 +49,7 @@ public class CapacitySchedulerQueueInfo {
   protected QueueState state;
   protected CapacitySchedulerQueueInfoList queues;
   protected ResourceInfo resourcesUsed;
-  private boolean hideReservationQueues = true;
+  private boolean hideReservationQueues = false;
 
   CapacitySchedulerQueueInfo() {
   };
@@ -72,8 +72,8 @@ public class CapacitySchedulerQueueInfo {
     state = q.getState();
     resourcesUsed = new ResourceInfo(q.getUsedResources());
     if(q instanceof PlanQueue &&
-       ((PlanQueue)q).showReservationsAsQueues()) {
-      hideReservationQueues = false;
+       !((PlanQueue)q).showReservationsAsQueues()) {
+      hideReservationQueues = true;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.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/applicationsmanager/MockAsm.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
index 7d92827..800f65b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -177,6 +178,11 @@ public abstract class MockAsm extends MockApps {
     public RMAppMetrics getRMAppMetrics() {
       return new RMAppMetrics(Resource.newInstance(0, 0), 0, 0, 0, 0);
     }
+
+    @Override
+    public ReservationId getReservationId() {
+      throw new UnsupportedOperationException("Not supported yet.");
+    }
   }
 
   public static RMApp newApplication(int i) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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 cbca6dc..71b5b8b 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
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 static org.mockito.Mockito.doReturn;
@@ -80,7 +97,7 @@ public class ReservationSystemTestUtil {
             + CapacitySchedulerConfiguration.DOT + reservationQ;
     conf.setCapacity(dedicated, 80);
     // Set as reservation queue
-    conf.setReservableQueue(dedicated, true);
+    conf.setReservable(dedicated, true);
 
     // Define 2nd-level queues
     final String A1 = A + ".a1";
@@ -118,11 +135,11 @@ public class ReservationSystemTestUtil {
     final String dedicated = prefix + reservationQ;
     conf.setCapacity(dedicated, 80);
     // Set as reservation queue
-    conf.setReservableQueue(dedicated, true);
+    conf.setReservable(dedicated, true);
 
     conf.setCapacity(prefix + newQ, 10);
     // Set as reservation queue
-    conf.setReservableQueue(prefix + newQ, true);
+    conf.setReservable(prefix + newQ, true);
 
     // Define 2nd-level queues
     final String A1 = A + ".a1";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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 83d6d3f..451a155 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
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 static org.junit.Assert.assertTrue;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityReservationSystem.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/TestCapacityReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityReservationSystem.java
index 2a77791..dd68277 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityReservationSystem.java
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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 0b0201d..f1e1e5c 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
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 static org.junit.Assert.assertFalse;
@@ -8,7 +25,6 @@ import static org.mockito.Mockito.mock;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
@@ -62,7 +78,7 @@ public class TestGreedyReservationAgent {
     capConf.setMaximumCapacity(reservationQ, 100);
     capConf.setAverageCapacity(reservationQ, 100);
     CapacityOverTimePolicy policy = new CapacityOverTimePolicy();
-    policy.init(reservationQ, capConf, new HashSet<String>());
+    policy.init(reservationQ, capConf);
     agent = new GreedyReservationAgent();
 
     QueueMetrics queueMetrics = QueueMetrics.forQueue("dedicated",
@@ -538,7 +554,7 @@ public class TestGreedyReservationAgent {
     capConf.setMaximumCapacity(reservationQ, 100);
     capConf.setAverageCapacity(reservationQ, 100);
     CapacityOverTimePolicy policy = new CapacityOverTimePolicy();
-    policy.init(reservationQ, capConf, new HashSet<String>());
+    policy.init(reservationQ, capConf);
 
     plan = new InMemoryPlan(scheduler.getRootQueueMetrics(), policy, agent,
       clusterCapacity, step, res, minAlloc, maxAlloc, "dedicated", null, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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 6dcd41f..91c1962 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
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 static org.mockito.Mockito.mock;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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 f4c4581..76f39dc 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
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 java.util.Collections;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.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/TestNoOverCommitPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.java
index 2ceead3..9389f12 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.java
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 static org.junit.Assert.assertTrue;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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 ab0de6b..c7301c7 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
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 java.util.HashMap;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.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/TestReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
index f5917bb..93adf74 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 static org.mockito.Matchers.any;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/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 f2313e6..c94ef69 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
@@ -1,3 +1,20 @@
+/*******************************************************************************
+ *   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 static org.junit.Assert.assertNotNull;
@@ -7,7 +24,6 @@ import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import java.util.HashSet;
 import java.util.Map;
 import java.util.TreeMap;
 
@@ -37,7 +53,7 @@ public class TestSimpleCapacityReplanner {
     ReservationAgent agent = mock(ReservationAgent.class);
 
     SharingPolicy policy = new NoOverCommitPolicy();
-    policy.init("root.dedicated", null, new HashSet<String>());
+    policy.init("root.dedicated", null);
 
     QueueMetrics queueMetrics = mock(QueueMetrics.class);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.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/rmapp/MockRMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
index 2fff718..787b5d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
@@ -258,4 +259,9 @@ public class MockRMApp implements RMApp {
   public RMAppMetrics getRMAppMetrics() {
     throw new UnsupportedOperationException("Not supported yet.");
   }
+
+  @Override
+  public ReservationId getReservationId() {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e10a13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.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/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.java
index aecbfa8..73d8a55 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.java
@@ -66,7 +66,7 @@ public class TestCapacitySchedulerDynamicBehavior {
     setupPlanQueueConfiguration(conf);
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
         ResourceScheduler.class);
-    conf.setBoolean(YarnConfiguration.RM_RESERVATIONS_ENABLE, false);
+    conf.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE, false);
     rm = new MockRM(conf);
     rm.start();
   }
@@ -272,7 +272,7 @@ public class TestCapacitySchedulerDynamicBehavior {
     conf.setCapacity(B3, B3_CAPACITY);
     conf.setUserLimitFactor(B3, 100.0f);
 
-    conf.setReservableQueue(A, true);
+    conf.setReservable(A, true);
     conf.setReservationWindow(A, 86400 * 1000);
     conf.setAverageCapacity(A, 1.0f);
 


[08/16] git commit: YARN-1711. Policy to enforce instantaneous and over-time quotas on user reservation. Contributed by Carlo Curino and Subru Krishnan. (cherry picked from commit c4918cb4cb5a267a8cfd6eace28fcfe7ad6174e8)

Posted by cd...@apache.org.
YARN-1711. Policy to enforce instantaneous and over-time quotas on user reservation. Contributed by Carlo Curino and Subru Krishnan.
(cherry picked from commit c4918cb4cb5a267a8cfd6eace28fcfe7ad6174e8)


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

Branch: refs/heads/trunk
Commit: b6df0dddcdafd7ec67c76ea92aea3ff3e94db247
Parents: f66ffcf
Author: carlo curino <Carlo Curino>
Authored: Tue Sep 16 13:20:57 2014 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Oct 3 15:42:03 2014 -0700

----------------------------------------------------------------------
 YARN-1051-CHANGES.txt                           |   3 +
 .../reservation/CapacityOverTimePolicy.java     | 231 +++++++++++++++++++
 .../reservation/NoOverCommitPolicy.java         |  74 ++++++
 .../reservation/SharingPolicy.java              |  49 ++++
 .../exceptions/ContractValidationException.java |   9 +-
 .../exceptions/MismatchedUserException.java     |  28 +++
 .../exceptions/PlanningException.java           |   9 +-
 .../exceptions/PlanningQuotaException.java      |  28 +++
 .../exceptions/ResourceOverCommitException.java |  28 +++
 .../CapacitySchedulerConfiguration.java         | 154 +++++++++++++
 .../reservation/TestCapacityOverTimePolicy.java | 222 ++++++++++++++++++
 .../reservation/TestNoOverCommitPolicy.java     | 144 ++++++++++++
 12 files changed, 977 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6df0ddd/YARN-1051-CHANGES.txt
----------------------------------------------------------------------
diff --git a/YARN-1051-CHANGES.txt b/YARN-1051-CHANGES.txt
index deece7c..e9ec691 100644
--- a/YARN-1051-CHANGES.txt
+++ b/YARN-1051-CHANGES.txt
@@ -14,3 +14,6 @@ subru)
 YARN-1710. Logic to find allocations within a Plan that satisfy 
 user ReservationRequest(s). (Carlo Curino and Subru Krishnan via 
 curino) 
+
+YARN-1711. Policy to enforce instantaneous and over-time quotas 
+on user reservations. (Carlo Curino and Subru Krishnan via curino)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6df0ddd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.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/CapacityOverTimePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.java
new file mode 100644
index 0000000..38c0207
--- /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/CapacityOverTimePolicy.java
@@ -0,0 +1,231 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import java.util.Date;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.MismatchedUserException;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningQuotaException;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.ResourceOverCommitException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+/**
+ * This policy enforces a time-extended notion of Capacity. In particular it
+ * guarantees that the allocation received in input when combined with all
+ * previous allocation for the user does not violate an instantaneous max limit
+ * on the resources received, and that for every window of time of length
+ * validWindow, the integral of the allocations for a user (sum of the currently
+ * submitted allocation and all prior allocations for the user) does not exceed
+ * validWindow * maxAvg.
+ * 
+ * This allows flexibility, in the sense that an allocation can instantaneously
+ * use large portions of the available capacity, but prevents abuses by bounding
+ * the average use over time.
+ * 
+ * By controlling maxInst, maxAvg, validWindow the administrator configuring
+ * this policy can obtain a behavior ranging from instantaneously enforced
+ * capacity (akin to existing queues), or fully flexible allocations (likely
+ * reserved to super-users, or trusted systems).
+ */
+@LimitedPrivate("yarn")
+@Unstable
+public class CapacityOverTimePolicy implements SharingPolicy {
+
+  private CapacitySchedulerConfiguration conf;
+  private long validWindow;
+  private float maxInst;
+  private float maxAvg;
+
+  // For now this is CapacityScheduler specific, but given a hierarchy in the
+  // configuration structure of the schedulers (e.g., SchedulerConfiguration)
+  // it should be easy to remove this limitation
+  @Override
+  public void init(String reservationQueuePath, Configuration conf) {
+    this.conf = (CapacitySchedulerConfiguration) conf;
+    validWindow = this.conf.getReservationWindow(reservationQueuePath);
+    maxInst = this.conf.getInstantaneousMaxCapacity(reservationQueuePath) / 100;
+    maxAvg = this.conf.getAverageCapacity(reservationQueuePath) / 100;
+  };
+
+  @Override
+  public void validate(Plan plan, ReservationAllocation reservation)
+      throws PlanningException {
+
+    // this is entire method invoked under a write-lock on the plan, no need
+    // to synchronize accesses to the plan further
+
+    // Try to verify whether there is already a reservation with this ID in
+    // the system (remove its contribution during validation to simulate a
+    // try-n-swap
+    // update).
+    ReservationAllocation oldReservation =
+        plan.getReservationById(reservation.getReservationId());
+
+    // sanity check that the update of a reservation is not changing username
+    if (oldReservation != null
+        && !oldReservation.getUser().equals(reservation.getUser())) {
+      throw new MismatchedUserException(
+          "Updating an existing reservation with mismatched user:"
+              + oldReservation.getUser() + " != " + reservation.getUser());
+    }
+
+    long startTime = reservation.getStartTime();
+    long endTime = reservation.getEndTime();
+    long step = plan.getStep();
+
+    Resource planTotalCapacity = plan.getTotalCapacity();
+
+    Resource maxAvgRes = Resources.multiply(planTotalCapacity, maxAvg);
+    Resource maxInsRes = Resources.multiply(planTotalCapacity, maxInst);
+
+    // define variable that will store integral of resources (need diff class to
+    // avoid overflow issues for long/large allocations)
+    IntegralResource runningTot = new IntegralResource(0L, 0L);
+    IntegralResource maxAllowed = new IntegralResource(maxAvgRes);
+    maxAllowed.multiplyBy(validWindow / step);
+
+    // check that the resources offered to the user during any window of length
+    // "validWindow" overlapping this allocation are within maxAllowed
+    // also enforce instantaneous and physical constraints during this pass
+    for (long t = startTime - validWindow; t < endTime + validWindow; t += step) {
+
+      Resource currExistingAllocTot = plan.getTotalCommittedResources(t);
+      Resource currExistingAllocForUser =
+          plan.getConsumptionForUser(reservation.getUser(), t);
+      Resource currNewAlloc = reservation.getResourcesAtTime(t);
+      Resource currOldAlloc = Resources.none();
+      if (oldReservation != null) {
+        currOldAlloc = oldReservation.getResourcesAtTime(t);
+      }
+
+      // throw exception if the cluster is overcommitted
+      // tot_allocated - old + new > capacity
+      Resource inst =
+          Resources.subtract(Resources.add(currExistingAllocTot, currNewAlloc),
+              currOldAlloc);
+      if (Resources.greaterThan(plan.getResourceCalculator(),
+          planTotalCapacity, inst, planTotalCapacity)) {
+        throw new ResourceOverCommitException(" Resources at time " + t
+            + " would be overcommitted (" + inst + " over "
+            + plan.getTotalCapacity() + ") by accepting reservation: "
+            + reservation.getReservationId());
+      }
+
+      // throw exception if instantaneous limits are violated
+      // tot_alloc_to_this_user - old + new > inst_limit
+      if (Resources.greaterThan(plan.getResourceCalculator(),
+          planTotalCapacity, Resources.subtract(
+              Resources.add(currExistingAllocForUser, currNewAlloc),
+              currOldAlloc), maxInsRes)) {
+        throw new PlanningQuotaException("Instantaneous quota capacity "
+            + maxInst + " would be passed at time " + t
+            + " by accepting reservation: " + reservation.getReservationId());
+      }
+
+      // throw exception if the running integral of utilization over validWindow
+      // is violated. We perform a delta check, adding/removing instants at the
+      // boundary of the window from runningTot.
+
+      // runningTot = previous_runningTot + currExistingAllocForUser +
+      // currNewAlloc - currOldAlloc - pastNewAlloc - pastOldAlloc;
+
+      // Where:
+      // 1) currNewAlloc, currExistingAllocForUser represent the contribution of
+      // the instant in time added in this pass.
+      // 2) pastNewAlloc, pastOldAlloc are the contributions relative to time
+      // instants that are being retired from the the window
+      // 3) currOldAlloc is the contribution (if any) of the previous version of
+      // this reservation (the one we are updating)
+
+      runningTot.add(currExistingAllocForUser);
+      runningTot.add(currNewAlloc);
+      runningTot.subtract(currOldAlloc);
+
+      // expire contributions from instant in time before (t - validWindow)
+      if (t > startTime) {
+        Resource pastOldAlloc =
+            plan.getConsumptionForUser(reservation.getUser(), t - validWindow);
+        Resource pastNewAlloc = reservation.getResourcesAtTime(t - validWindow);
+
+        // runningTot = runningTot - pastExistingAlloc - pastNewAlloc;
+        runningTot.subtract(pastOldAlloc);
+        runningTot.subtract(pastNewAlloc);
+      }
+
+      // check integral
+      // runningTot > maxAvg * validWindow
+      // NOTE: we need to use comparator of IntegralResource directly, as
+      // Resource and ResourceCalculator assume "int" amount of resources,
+      // which is not sufficient when comparing integrals (out-of-bound)
+      if (maxAllowed.compareTo(runningTot) < 0) {
+        throw new PlanningQuotaException(
+            "Integral (avg over time) quota capacity " + maxAvg
+                + " over a window of " + validWindow / 1000 + " seconds, "
+                + " would be passed at time " + t + "(" + new Date(t)
+                + ") by accepting reservation: "
+                + reservation.getReservationId());
+      }
+    }
+  }
+
+  @Override
+  public long getValidWindow() {
+    return validWindow;
+  }
+
+  /**
+   * This class provides support for Resource-like book-keeping, based on
+   * long(s), as using Resource to store the "integral" of the allocation over
+   * time leads to integer overflows for large allocations/clusters. (Evolving
+   * Resource to use long is too disruptive at this point.)
+   * 
+   * The comparison/multiplication behaviors of IntegralResource are consistent
+   * with the DefaultResourceCalculator.
+   */
+  public class IntegralResource {
+    long memory;
+    long vcores;
+
+    public IntegralResource(Resource resource) {
+      this.memory = resource.getMemory();
+      this.vcores = resource.getVirtualCores();
+    }
+
+    public IntegralResource(long mem, long vcores) {
+      this.memory = mem;
+      this.vcores = vcores;
+    }
+
+    public void add(Resource r) {
+      memory += r.getMemory();
+      vcores += r.getVirtualCores();
+    }
+
+    public void subtract(Resource r) {
+      memory -= r.getMemory();
+      vcores -= r.getVirtualCores();
+    }
+
+    public void multiplyBy(long window) {
+      memory = memory * window;
+      vcores = vcores * window;
+    }
+
+    public long compareTo(IntegralResource other) {
+      long diff = memory - other.memory;
+      if (diff == 0) {
+        diff = vcores - other.vcores;
+      }
+      return diff;
+    }
+
+    @Override
+    public String toString() {
+      return "<memory:" + memory + ", vCores:" + vcores + ">";
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6df0ddd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.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/NoOverCommitPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
new file mode 100644
index 0000000..cbe2b78
--- /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/NoOverCommitPolicy.java
@@ -0,0 +1,74 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.MismatchedUserException;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.ResourceOverCommitException;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+/**
+ * This policy enforce a simple physical cluster capacity constraints, by
+ * validating that the allocation proposed fits in the current plan. This
+ * validation is compatible with "updates" and in verifying the capacity
+ * constraints it conceptually remove the prior version of the reservation.
+ */
+@LimitedPrivate("yarn")
+@Unstable
+public class NoOverCommitPolicy implements SharingPolicy {
+
+  @Override
+  public void validate(Plan plan, ReservationAllocation reservation)
+      throws PlanningException {
+
+    ReservationAllocation oldReservation =
+        plan.getReservationById(reservation.getReservationId());
+
+    // check updates are using same name
+    if (oldReservation != null
+        && !oldReservation.getUser().equals(reservation.getUser())) {
+      throw new MismatchedUserException(
+          "Updating an existing reservation with mismatching user:"
+              + oldReservation.getUser() + " != " + reservation.getUser());
+    }
+
+    long startTime = reservation.getStartTime();
+    long endTime = reservation.getEndTime();
+    long step = plan.getStep();
+
+    // for every instant in time, check we are respecting cluster capacity
+    for (long t = startTime; t < endTime; t += step) {
+      Resource currExistingAllocTot = plan.getTotalCommittedResources(t);
+      Resource currNewAlloc = reservation.getResourcesAtTime(t);
+      Resource currOldAlloc = Resource.newInstance(0, 0);
+      if (oldReservation != null) {
+        oldReservation.getResourcesAtTime(t);
+      }
+      // check the cluster is never over committed
+      // currExistingAllocTot + currNewAlloc - currOldAlloc >
+      // capPlan.getTotalCapacity()
+      if (Resources.greaterThan(plan.getResourceCalculator(), plan
+          .getTotalCapacity(), Resources.subtract(
+          Resources.add(currExistingAllocTot, currNewAlloc), currOldAlloc),
+          plan.getTotalCapacity())) {
+        throw new ResourceOverCommitException("Resources at time " + t
+            + " would be overcommitted by " + "accepting reservation: "
+            + reservation.getReservationId());
+      }
+    }
+  }
+
+  @Override
+  public long getValidWindow() {
+    // this policy has no "memory" so the valid window is set to zero
+    return 0;
+  }
+
+  @Override
+  public void init(String inventoryQueuePath, Configuration conf) {
+    // nothing to do for this policy
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6df0ddd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.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/SharingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SharingPolicy.java
new file mode 100644
index 0000000..d917764
--- /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/SharingPolicy.java
@@ -0,0 +1,49 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+
+/**
+ * This is the interface for policy that validate new
+ * {@link ReservationAllocation}s for allocations being added to a {@link Plan}.
+ * Individual policies will be enforcing different invariants.
+ */
+@LimitedPrivate("yarn")
+@Unstable
+public interface SharingPolicy {
+
+  /**
+   * Initialize this policy
+   * 
+   * @param inventoryQueuePath the name of the queue for this plan
+   * @param conf the system configuration
+   */
+  public void init(String inventoryQueuePath, Configuration conf);
+
+  /**
+   * This method runs the policy validation logic, and return true/false on
+   * whether the {@link ReservationAllocation} is acceptable according to this
+   * sharing policy.
+   * 
+   * @param plan the {@link Plan} we validate against
+   * @param newAllocation the allocation proposed to be added to the
+   *          {@link Plan}
+   * @throws PlanningException if the policy is respected if we add this
+   *           {@link ReservationAllocation} to the {@link Plan}
+   */
+  public void validate(Plan plan, ReservationAllocation newAllocation)
+      throws PlanningException;
+
+  /**
+   * Returns the time range before and after the current reservation considered
+   * by this policy. In particular, this informs the archival process for the
+   * {@link Plan}, i.e., reservations regarding times before (now - validWindow)
+   * can be deleted.
+   * 
+   * @return validWindow the window of validity considered by the policy.
+   */
+  public long getValidWindow();
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6df0ddd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ContractValidationException.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/exceptions/ContractValidationException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ContractValidationException.java
index 7ee5a76..cd82a9e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ContractValidationException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ContractValidationException.java
@@ -1,5 +1,13 @@
 package org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions;
 
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * This exception is thrown if the request made is not syntactically valid.
+ */
+@Public
+@Unstable
 public class ContractValidationException extends PlanningException {
 
   private static final long serialVersionUID = 1L;
@@ -8,5 +16,4 @@ public class ContractValidationException extends PlanningException {
     super(message);
   }
 
-  
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6df0ddd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/MismatchedUserException.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/exceptions/MismatchedUserException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/MismatchedUserException.java
new file mode 100644
index 0000000..0a443f3
--- /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/exceptions/MismatchedUserException.java
@@ -0,0 +1,28 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Exception thrown when an update to an existing reservation is performed
+ * by a user that is not the reservation owner. 
+ */
+@Public
+@Unstable
+public class MismatchedUserException extends PlanningException {
+
+  private static final long serialVersionUID = 8313222590561668413L;
+
+  public MismatchedUserException(String message) {
+    super(message);
+  }
+
+  public MismatchedUserException(Throwable cause) {
+    super(cause);
+  }
+
+  public MismatchedUserException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6df0ddd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningException.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/exceptions/PlanningException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningException.java
index aa9e9fb..0699856 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningException.java
@@ -2,10 +2,17 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions;
 
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
 
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
 /**
  * Exception thrown by the admission control subsystem when there is a problem
- * in trying to find an allocation for a user {@link ReservationSubmissionRequest}.
+ * in trying to find an allocation for a user
+ * {@link ReservationSubmissionRequest}.
  */
+
+@Public
+@Unstable
 public class PlanningException extends Exception {
 
   private static final long serialVersionUID = -684069387367879218L;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6df0ddd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningQuotaException.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/exceptions/PlanningQuotaException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningQuotaException.java
new file mode 100644
index 0000000..aad4ee8
--- /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/exceptions/PlanningQuotaException.java
@@ -0,0 +1,28 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * This exception is thrown if the user quota is exceed while accepting or
+ * updating a reservation.
+ */
+@Public
+@Unstable
+public class PlanningQuotaException extends PlanningException {
+
+  private static final long serialVersionUID = 8206629288380246166L;
+
+  public PlanningQuotaException(String message) {
+    super(message);
+  }
+
+  public PlanningQuotaException(Throwable cause) {
+    super(cause);
+  }
+
+  public PlanningQuotaException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6df0ddd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ResourceOverCommitException.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/exceptions/ResourceOverCommitException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ResourceOverCommitException.java
new file mode 100644
index 0000000..a4c2b07
--- /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/exceptions/ResourceOverCommitException.java
@@ -0,0 +1,28 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * This exception indicate that the reservation that has been attempted, would
+ * exceed the physical resources available in the {@link Plan} at the moment.
+ */
+@Public
+@Unstable
+public class ResourceOverCommitException extends PlanningException {
+
+  private static final long serialVersionUID = 7070699407526521032L;
+
+  public ResourceOverCommitException(String message) {
+    super(message);
+  }
+
+  public ResourceOverCommitException(Throwable cause) {
+    super(cause);
+  }
+
+  public ResourceOverCommitException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6df0ddd/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 5542ef3..ba501b6 100644
--- a/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
+++ b/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
@@ -190,6 +190,63 @@ public class CapacitySchedulerConfiguration extends Configuration {
     }
   }
   
+  @Private
+  public static final String AVERAGE_CAPACITY = "average-capacity";
+
+  @Private
+  public static final String IS_RESERVABLE = "reservable";
+
+  @Private
+  public static final String RESERVATION_WINDOW = "reservation-window";
+
+  @Private
+  public static final String INSTANTANEOUS_MAX_CAPACITY =
+      "instantaneous-max-capacity";
+
+  @Private
+  public static final long DEFAULT_RESERVATION_WINDOW = 0L;
+
+  @Private
+  public static final String RESERVATION_ADMISSION_POLICY =
+      "reservation-policy";
+
+  @Private
+  public static final String RESERVATION_AGENT_NAME = "reservation-agent";
+
+  @Private
+  public static final String RESERVATION_SHOW_RESERVATION_AS_QUEUE =
+      "show-reservations-as-queues";
+
+  @Private
+  public static final String DEFAULT_RESERVATION_ADMISSION_POLICY =
+      "org.apache.hadoop.yarn.server.resourcemanager.reservation.CapacityOverTimePolicy";
+
+  @Private
+  public static final String DEFAULT_RESERVATION_AGENT_NAME =
+      "org.apache.hadoop.yarn.server.resourcemanager.reservation.GreedyReservationAgent";
+
+  @Private
+  public static final String RESERVATION_PLANNER_NAME = "reservation-planner";
+
+  @Private
+  public static final String DEFAULT_RESERVATION_PLANNER_NAME =
+      "org.apache.hadoop.yarn.server.resourcemanager.reservation.SimpleCapacityReplanner";
+
+  @Private
+  public static final String RESERVATION_MOVE_ON_EXPIRY =
+      "reservation-move-on-expiry";
+
+  @Private
+  public static final boolean DEFAULT_RESERVATION_MOVE_ON_EXPIRY = true;
+
+  @Private
+  public static final String RESERVATION_ENFORCEMENT_WINDOW =
+      "reservation-enforcement-window";
+
+  // default to 1h lookahead enforcement
+  @Private
+  public static final long DEFAULT_RESERVATION_ENFORCEMENT_WINDOW = 3600000;
+
   public CapacitySchedulerConfiguration() {
     this(new Configuration());
   }
@@ -511,4 +568,101 @@ public class CapacitySchedulerConfiguration extends Configuration {
 
     return mappings;
   }
+
+  public boolean isReservable(String queue) {
+    boolean isReservable =
+        getBoolean(getQueuePrefix(queue) + IS_RESERVABLE, false);
+    return isReservable;
+  }
+
+  public void setReservable(String queue, boolean isReservable) {
+    setBoolean(getQueuePrefix(queue) + IS_RESERVABLE, isReservable);
+    LOG.debug("here setReservableQueue: queuePrefix=" + getQueuePrefix(queue)
+        + ", isReservableQueue=" + isReservable(queue));
+  }
+
+  public long getReservationWindow(String queue) {
+    long reservationWindow =
+        getLong(getQueuePrefix(queue) + RESERVATION_WINDOW,
+            DEFAULT_RESERVATION_WINDOW);
+    return reservationWindow;
+  }
+
+  public float getAverageCapacity(String queue) {
+    float avgCapacity =
+        getFloat(getQueuePrefix(queue) + AVERAGE_CAPACITY,
+            MAXIMUM_CAPACITY_VALUE);
+    return avgCapacity;
+  }
+
+  public float getInstantaneousMaxCapacity(String queue) {
+    float instMaxCapacity =
+        getFloat(getQueuePrefix(queue) + INSTANTANEOUS_MAX_CAPACITY,
+            MAXIMUM_CAPACITY_VALUE);
+    return instMaxCapacity;
+  }
+
+  public void setInstantaneousMaxCapacity(String queue, float instMaxCapacity) {
+    setFloat(getQueuePrefix(queue) + INSTANTANEOUS_MAX_CAPACITY,
+        instMaxCapacity);
+  }
+
+  public void setReservationWindow(String queue, long reservationWindow) {
+    setLong(getQueuePrefix(queue) + RESERVATION_WINDOW, reservationWindow);
+  }
+
+  public void setAverageCapacity(String queue, float avgCapacity) {
+    setFloat(getQueuePrefix(queue) + AVERAGE_CAPACITY, avgCapacity);
+  }
+
+  public String getReservationAdmissionPolicy(String queue) {
+    String reservationPolicy =
+        get(getQueuePrefix(queue) + RESERVATION_ADMISSION_POLICY,
+            DEFAULT_RESERVATION_ADMISSION_POLICY);
+    return reservationPolicy;
+  }
+
+  public void setReservationAdmissionPolicy(String queue,
+      String reservationPolicy) {
+    set(getQueuePrefix(queue) + RESERVATION_ADMISSION_POLICY, reservationPolicy);
+  }
+
+  public String getReservationAgent(String queue) {
+    String reservationAgent =
+        get(getQueuePrefix(queue) + RESERVATION_AGENT_NAME,
+            DEFAULT_RESERVATION_AGENT_NAME);
+    return reservationAgent;
+  }
+
+  public void setReservationAgent(String queue, String reservationPolicy) {
+    set(getQueuePrefix(queue) + RESERVATION_AGENT_NAME, reservationPolicy);
+  }
+
+  public boolean getShowReservationAsQueues(String queuePath) {
+    boolean showReservationAsQueues =
+        getBoolean(getQueuePrefix(queuePath)
+            + RESERVATION_SHOW_RESERVATION_AS_QUEUE, false);
+    return showReservationAsQueues;
+  }
+
+  public String getReplanner(String queue) {
+    String replanner =
+        get(getQueuePrefix(queue) + RESERVATION_PLANNER_NAME,
+            DEFAULT_RESERVATION_PLANNER_NAME);
+    return replanner;
+  }
+
+  public boolean getMoveOnExpiry(String queue) {
+    boolean killOnExpiry =
+        getBoolean(getQueuePrefix(queue) + RESERVATION_MOVE_ON_EXPIRY,
+            DEFAULT_RESERVATION_MOVE_ON_EXPIRY);
+    return killOnExpiry;
+  }
+
+  public long getEnforcementWindow(String queue) {
+    long enforcementWindow =
+        getLong(getQueuePrefix(queue) + RESERVATION_ENFORCEMENT_WINDOW,
+            DEFAULT_RESERVATION_ENFORCEMENT_WINDOW);
+    return enforcementWindow;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6df0ddd/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
new file mode 100644
index 0000000..83d6d3f
--- /dev/null
+++ 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
@@ -0,0 +1,222 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+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.reservation.exceptions.PlanningQuotaException;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.ResourceOverCommitException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCapacityOverTimePolicy {
+
+  long timeWindow;
+  long step;
+  float avgConstraint;
+  float instConstraint;
+  long initTime;
+
+  InMemoryPlan plan;
+  ReservationAgent mAgent;
+  Resource minAlloc;
+  ResourceCalculator res;
+  Resource maxAlloc;
+
+  int totCont = 1000000;
+
+  @Before
+  public void setup() throws Exception {
+
+    // 24h window
+    timeWindow = 86400000L;
+    // 1 sec step
+    step = 1000L;
+
+    // 25% avg cap on capacity
+    avgConstraint = 25;
+
+    // 70% instantaneous cap on capacity
+    instConstraint = 70;
+
+    initTime = System.currentTimeMillis();
+    minAlloc = Resource.newInstance(1024, 1);
+    res = new DefaultResourceCalculator();
+    maxAlloc = Resource.newInstance(1024 * 8, 8);
+
+    mAgent = mock(ReservationAgent.class);
+    ReservationSystemTestUtil testUtil = new ReservationSystemTestUtil();
+    CapacityScheduler scheduler = testUtil.mockCapacityScheduler(totCont);
+    String reservationQ = testUtil.getFullReservationQueueName();
+    CapacitySchedulerConfiguration capConf = scheduler.getConfiguration();
+    capConf.setReservationWindow(reservationQ, timeWindow);
+    capConf.setInstantaneousMaxCapacity(reservationQ, instConstraint);
+    capConf.setAverageCapacity(reservationQ, avgConstraint);
+    CapacityOverTimePolicy policy = new CapacityOverTimePolicy();
+    policy.init(reservationQ, capConf);
+
+    plan =
+        new InMemoryPlan(scheduler.getRootQueueMetrics(), policy, mAgent,
+            scheduler.getClusterResource(), step, res, minAlloc, maxAlloc,
+            "dedicated", null, true);
+  }
+
+  public int[] generateData(int length, int val) {
+    int[] data = new int[length];
+    for (int i = 0; i < length; i++) {
+      data[i] = val;
+    }
+    return data;
+  }
+
+  @Test
+  public void testSimplePass() throws IOException, PlanningException {
+    // generate allocation that simply fit within all constraints
+    int[] f = generateData(3600, (int) Math.ceil(0.2 * totCont));
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(
+            ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+            "dedicated", initTime, initTime + f.length,
+            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
+            res, minAlloc)));
+  }
+
+  @Test
+  public void testSimplePass2() throws IOException, PlanningException {
+    // generate allocation from single tenant that exceed avg momentarily but
+    // fit within
+    // max instantanesou
+    int[] f = generateData(3600, (int) Math.ceil(0.69 * totCont));
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(
+            ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+            "dedicated", initTime, initTime + f.length,
+            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
+            res, minAlloc)));
+  }
+
+  @Test
+  public void testMultiTenantPass() throws IOException, PlanningException {
+    // generate allocation from multiple tenants that barely fit in tot capacity
+    int[] f = generateData(3600, (int) Math.ceil(0.25 * totCont));
+    for (int i = 0; i < 4; i++) {
+      assertTrue(plan.toString(),
+          plan.addReservation(new InMemoryReservationAllocation(
+              ReservationSystemTestUtil.getNewReservationId(), null, "u" + i,
+              "dedicated", initTime, initTime + f.length,
+              ReservationSystemTestUtil.generateAllocation(initTime, step, f),
+              res, minAlloc)));
+    }
+  }
+
+  @Test(expected = ResourceOverCommitException.class)
+  public void testMultiTenantFail() throws IOException, PlanningException {
+    // generate allocation from multiple tenants that exceed tot capacity
+    int[] f = generateData(3600, (int) Math.ceil(0.25 * totCont));
+    for (int i = 0; i < 5; i++) {
+      assertTrue(plan.toString(),
+          plan.addReservation(new InMemoryReservationAllocation(
+              ReservationSystemTestUtil.getNewReservationId(), null, "u" + i,
+              "dedicated", initTime, initTime + f.length,
+              ReservationSystemTestUtil.generateAllocation(initTime, step, f),
+              res, minAlloc)));
+    }
+  }
+
+  @Test(expected = PlanningQuotaException.class)
+  public void testInstFail() throws IOException, PlanningException {
+    // generate allocation that exceed the instantaneous cap single-show
+    int[] f = generateData(3600, (int) Math.ceil(0.71 * totCont));
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(
+            ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+            "dedicated", initTime, initTime + f.length,
+            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
+            res, minAlloc)));
+    Assert.fail("should not have accepted this");
+  }
+
+  @Test
+  public void testInstFailBySum() throws IOException, PlanningException {
+    // generate allocation that exceed the instantaneous cap by sum
+    int[] f = generateData(3600, (int) Math.ceil(0.3 * totCont));
+
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(
+            ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+            "dedicated", initTime, initTime + f.length,
+            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
+            res, minAlloc)));
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(
+            ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+            "dedicated", initTime, initTime + f.length,
+            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
+            res, minAlloc)));
+    try {
+      assertTrue(plan.toString(),
+          plan.addReservation(new InMemoryReservationAllocation(
+              ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+              "dedicated", initTime, initTime + f.length,
+              ReservationSystemTestUtil.generateAllocation(initTime, step, f),
+              res, minAlloc)));
+      Assert.fail();
+    } catch (PlanningQuotaException p) {
+      // expected
+    }
+  }
+
+  @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>();
+    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));
+
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(
+            ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+            "dedicated", initTime, initTime + win, req, res, minAlloc)));
+  }
+
+  @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>();
+    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));
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(
+            ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+            "dedicated", initTime, initTime + win, req, res, minAlloc)));
+
+    try {
+      assertTrue(plan.toString(),
+          plan.addReservation(new InMemoryReservationAllocation(
+              ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+              "dedicated", initTime, initTime + win, req, res, minAlloc)));
+
+      Assert.fail("should not have accepted this");
+    } catch (PlanningQuotaException e) {
+      // expected
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6df0ddd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.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/TestNoOverCommitPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.java
new file mode 100644
index 0000000..2ceead3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.java
@@ -0,0 +1,144 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+import java.io.IOException;
+
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.MismatchedUserException;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.ResourceOverCommitException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestNoOverCommitPolicy {
+
+  long step;
+  long initTime;
+
+  InMemoryPlan plan;
+  ReservationAgent mAgent;
+  Resource minAlloc;
+  ResourceCalculator res;
+  Resource maxAlloc;
+
+  int totCont = 1000000;
+
+  @Before
+  public void setup() throws Exception {
+
+    // 1 sec step
+    step = 1000L;
+
+    initTime = System.currentTimeMillis();
+    minAlloc = Resource.newInstance(1024, 1);
+    res = new DefaultResourceCalculator();
+    maxAlloc = Resource.newInstance(1024 * 8, 8);
+
+    mAgent = mock(ReservationAgent.class);
+    ReservationSystemTestUtil testUtil = new ReservationSystemTestUtil();
+    CapacityScheduler scheduler = testUtil.mockCapacityScheduler(totCont);
+    String reservationQ = testUtil.getFullReservationQueueName();
+    CapacitySchedulerConfiguration capConf = scheduler.getConfiguration();
+    NoOverCommitPolicy policy = new NoOverCommitPolicy();
+    policy.init(reservationQ, capConf);
+
+    plan =
+        new InMemoryPlan(scheduler.getRootQueueMetrics(), policy, mAgent,
+            scheduler.getClusterResource(), step, res, minAlloc, maxAlloc,
+            "dedicated", null, true);
+  }
+
+  public int[] generateData(int length, int val) {
+    int[] data = new int[length];
+    for (int i = 0; i < length; i++) {
+      data[i] = val;
+    }
+    return data;
+  }
+
+  @Test
+  public void testSingleUserEasyFitPass() throws IOException, PlanningException {
+    // generate allocation that easily fit within resource constraints
+    int[] f = generateData(3600, (int) Math.ceil(0.2 * totCont));
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(
+            ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+            "dedicated", initTime, initTime + f.length,
+            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
+            res, minAlloc)));
+  }
+
+  @Test
+  public void testSingleUserBarelyFitPass() throws IOException,
+      PlanningException {
+    // generate allocation from single tenant that barely fit
+    int[] f = generateData(3600, totCont);
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(
+            ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+            "dedicated", initTime, initTime + f.length,
+            ReservationSystemTestUtil.generateAllocation(initTime, step, f),
+            res, minAlloc)));
+  }
+
+  @Test(expected = ResourceOverCommitException.class)
+  public void testSingleFail() throws IOException, PlanningException {
+    // generate allocation from single tenant that exceed capacity
+    int[] f = generateData(3600, (int) (1.1 * totCont));
+    plan.addReservation(new InMemoryReservationAllocation(
+        ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+        "dedicated", initTime, initTime + f.length, ReservationSystemTestUtil
+            .generateAllocation(initTime, step, f), res, minAlloc));
+  }
+
+  @Test(expected = MismatchedUserException.class)
+  public void testUserMismatch() throws IOException, PlanningException {
+    // generate allocation from single tenant that exceed capacity
+    int[] f = generateData(3600, (int) (0.5 * totCont));
+
+    ReservationId rid = ReservationSystemTestUtil.getNewReservationId();
+    plan.addReservation(new InMemoryReservationAllocation(rid, null, "u1",
+        "dedicated", initTime, initTime + f.length, ReservationSystemTestUtil
+            .generateAllocation(initTime, step, f), res, minAlloc));
+
+    // trying to update a reservation with a mismatching user
+    plan.updateReservation(new InMemoryReservationAllocation(rid, null, "u2",
+        "dedicated", initTime, initTime + f.length, ReservationSystemTestUtil
+            .generateAllocation(initTime, step, f), res, minAlloc));
+  }
+
+  @Test
+  public void testMultiTenantPass() throws IOException, PlanningException {
+    // generate allocation from multiple tenants that barely fit in tot capacity
+    int[] f = generateData(3600, (int) Math.ceil(0.25 * totCont));
+    for (int i = 0; i < 4; i++) {
+      assertTrue(plan.toString(),
+          plan.addReservation(new InMemoryReservationAllocation(
+              ReservationSystemTestUtil.getNewReservationId(), null, "u" + i,
+              "dedicated", initTime, initTime + f.length,
+              ReservationSystemTestUtil.generateAllocation(initTime, step, f),
+              res, minAlloc)));
+    }
+  }
+
+  @Test(expected = ResourceOverCommitException.class)
+  public void testMultiTenantFail() throws IOException, PlanningException {
+    // generate allocation from multiple tenants that exceed tot capacity
+    int[] f = generateData(3600, (int) Math.ceil(0.25 * totCont));
+    for (int i = 0; i < 5; i++) {
+      assertTrue(plan.toString(),
+          plan.addReservation(new InMemoryReservationAllocation(
+              ReservationSystemTestUtil.getNewReservationId(), null, "u" + i,
+              "dedicated", initTime, initTime + f.length,
+              ReservationSystemTestUtil.generateAllocation(initTime, step, f),
+              res, minAlloc)));
+    }
+  }
+}


[16/16] git commit: YARN-1051. Add a system for creating reservations of cluster capacity. Contributed by Subru Krishnan and Carlo Curino.

Posted by cd...@apache.org.
YARN-1051. Add a system for creating reservations of cluster capacity.
Contributed by Subru Krishnan and Carlo Curino.


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

Branch: refs/heads/trunk
Commit: c8212bacb1b2a7e6ee83cc56f72297465ce99390
Parents: a298623
Author: Chris Douglas <cd...@apache.org>
Authored: Fri Oct 3 17:05:57 2014 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Oct 3 17:09:02 2014 -0700

----------------------------------------------------------------------
 YARN-1051-CHANGES.txt                           | 34 -----------------
 hadoop-yarn-project/CHANGES.txt                 | 40 ++++++++++++++++++++
 .../src/main/proto/yarn_protos.proto            |  1 +
 .../pb/ApplicationSubmissionContextPBImpl.java  |  9 +++++
 .../yarn/server/resourcemanager/RMContext.java  |  1 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |  9 +++--
 .../scheduler/capacity/LeafQueue.java           |  2 +-
 .../scheduler/capacity/PlanQueue.java           |  3 +-
 8 files changed, 60 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8212bac/YARN-1051-CHANGES.txt
----------------------------------------------------------------------
diff --git a/YARN-1051-CHANGES.txt b/YARN-1051-CHANGES.txt
deleted file mode 100644
index 5cd1136..0000000
--- a/YARN-1051-CHANGES.txt
+++ /dev/null
@@ -1,34 +0,0 @@
-YARN-1707. Introduce APIs to add/remove/resize queues in the
-CapacityScheduler. (Carlo Curino and Subru Krishnan via curino)
-
-YARN-2475. Logic for responding to capacity drops for the 
-ReservationSystem. (Carlo Curino and Subru Krishnan via curino)
-
-YARN-1708. Public YARN APIs for creating/updating/deleting 
-reservations. (Subru Krishnan and Carlo Curino  via subru)
-
-YARN-1709. In-memory data structures used to track resources over
-time to enable reservations. (Subru Krishnan and Carlo Curino via 
-subru)
-
-YARN-1710. Logic to find allocations within a Plan that satisfy 
-user ReservationRequest(s). (Carlo Curino and Subru Krishnan via 
-curino) 
-
-YARN-1711. Policy to enforce instantaneous and over-time quotas 
-on user reservations. (Carlo Curino and Subru Krishnan via curino)
-
-YARN-1712. Plan follower that synchronizes the current state of reservation
-subsystem with the scheduler. (Subru Krishnan and Carlo Curino  via subru)
-
-YARN-2080. Integrating reservation system with ResourceManager and 
-client-RM protocol. (Subru Krishnan and Carlo Curino  via subru)
-
-MAPREDUCE-6103. Adding reservation APIs to MR resource manager
-delegate. (Subru Krishnan and Carlo Curino  via subru)
-
-YARN-2576. Fixing compilation, javadocs and audit issues to pass
-test patch in branch. (Subru Krishnan and Carlo Curino  via subru)
-
-YARN-2611. Fixing jenkins findbugs warning and TestRMWebServicesCapacitySched
-for branch YARN-1051. (Subru Krishnan and Carlo Curino  via subru)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8212bac/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 35c6cc0..8e4c5fe 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -6,6 +6,9 @@ Trunk - Unreleased
 
   NEW FEATURES
 
+    YARN-1051. Add a system for creating reservations of cluster capacity.
+    (see breakdown below)
+
   IMPROVEMENTS
 
     YARN-2438. yarn-env.sh cleanup (aw)
@@ -27,6 +30,43 @@ Trunk - Unreleased
 
     YARN-2525. yarn logs command gives error on trunk (Akira AJISAKA via aw)
 
+  BREAKDOWN OF YARN-1051 SUBTASKS AND RELATED JIRAS
+
+    YARN-1707. Introduce APIs to add/remove/resize queues in the
+    CapacityScheduler. (Carlo Curino and Subru Krishnan via curino)
+
+    YARN-2475. Logic for responding to capacity drops for the
+    ReservationSystem. (Carlo Curino and Subru Krishnan via curino)
+
+    YARN-1708. Public YARN APIs for creating/updating/deleting
+    reservations. (Subru Krishnan and Carlo Curino via subru)
+
+    YARN-1709. In-memory data structures used to track resources over
+    time to enable reservations. (Subru Krishnan and Carlo Curino via
+    subru)
+
+    YARN-1710. Logic to find allocations within a Plan that satisfy
+    user ReservationRequest(s). (Carlo Curino and Subru Krishnan via
+    curino)
+
+    YARN-1711. Policy to enforce instantaneous and over-time quotas
+    on user reservations. (Carlo Curino and Subru Krishnan via curino)
+
+    YARN-1712. Plan follower that synchronizes the current state of reservation
+    subsystem with the scheduler. (Subru Krishnan and Carlo Curino via subru)
+
+    YARN-2080. Integrating reservation system with ResourceManager and
+    client-RM protocol. (Subru Krishnan and Carlo Curino via subru)
+
+    MAPREDUCE-6103. Adding reservation APIs to MR resource manager
+    delegate. (Subru Krishnan and Carlo Curino via subru)
+
+    YARN-2576. Fixing compilation, javadocs and audit issues to pass
+    test patch in branch. (Subru Krishnan and Carlo Curino via subru)
+
+    YARN-2611. Fixing jenkins findbugs warning and TestRMWebServicesCapacitySched
+    for branch YARN-1051. (Subru Krishnan and Carlo Curino via subru)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8212bac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index c645719..d07ce13 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -202,6 +202,7 @@ message ApplicationAttemptReportProto {
   optional string diagnostics = 5 [default = "N/A"];
   optional YarnApplicationAttemptStateProto yarn_application_attempt_state = 6;
   optional ContainerIdProto am_container_id = 7;
+  optional string original_tracking_url = 8;
 }
 
 enum NodeStateProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8212bac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
index 489cf8c..9462a4e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
@@ -35,6 +36,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LogAggregationContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 
 import com.google.protobuf.TextFormat;
@@ -115,6 +117,13 @@ extends ApplicationSubmissionContext {
       builder.clearApplicationTags();
       builder.addAllApplicationTags(this.applicationTags);
     }
+    if (this.logAggregationContext != null) {
+      builder.setLogAggregationContext(
+          convertToProtoFormat(this.logAggregationContext));
+    }
+    if (this.reservationId != null) {
+      builder.setReservationId(convertToProtoFormat(this.reservationId));
+    }
   }
 
   private void mergeLocalToProto() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8212bac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.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/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index 46ecfcd..a59965f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8212bac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.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/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 84ec766..c0681aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -832,14 +832,16 @@ public class RMAppImpl implements RMApp, Recoverable {
       // started or started but not yet saved.
       if (app.attempts.isEmpty()) {
         app.scheduler.handle(new AppAddedSchedulerEvent(app.applicationId,
-          app.submissionContext.getQueue(), app.user));
+          app.submissionContext.getQueue(), app.user,
+          app.submissionContext.getReservationID()));
         return RMAppState.SUBMITTED;
       }
 
       // Add application to scheduler synchronously to guarantee scheduler
       // knows applications before AM or NM re-registers.
       app.scheduler.handle(new AppAddedSchedulerEvent(app.applicationId,
-        app.submissionContext.getQueue(), app.user, true));
+        app.submissionContext.getQueue(), app.user, true,
+          app.submissionContext.getReservationID()));
 
       // recover attempts
       app.recoverAppAttempts();
@@ -866,7 +868,8 @@ public class RMAppImpl implements RMApp, Recoverable {
     @Override
     public void transition(RMAppImpl app, RMAppEvent event) {
       app.handler.handle(new AppAddedSchedulerEvent(app.applicationId,
-        app.submissionContext.getQueue(), app.user));
+        app.submissionContext.getQueue(), app.user,
+        app.submissionContext.getReservationID()));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8212bac/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
----------------------------------------------------------------------
diff --git a/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 b/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
index b9f5d5f..f0cff71 100644
--- a/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
+++ b/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
@@ -225,7 +225,7 @@ public class LeafQueue implements CSQueue {
     return (float)scheduler.getConfiguration().getCapacity(getQueuePath()) / 100;
   }
 
-  private synchronized void setupQueueConfigs(
+  protected synchronized void setupQueueConfigs(
       Resource clusterResource,
       float capacity, float absoluteCapacity, 
       float maximumCapacity, float absoluteMaxCapacity,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8212bac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java
index abb8e95..b87744d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java
@@ -103,7 +103,8 @@ public class PlanQueue extends ParentQueue {
         newlyParsedParentQueue.getAbsoluteCapacity(),
         newlyParsedParentQueue.getMaximumCapacity(),
         newlyParsedParentQueue.getAbsoluteMaximumCapacity(),
-        newlyParsedParentQueue.getState(), newlyParsedParentQueue.getACLs());
+        newlyParsedParentQueue.getState(), newlyParsedParentQueue.getACLs(),
+        newlyParsedParentQueue.getReservationContinueLooking());
 
     updateQuotas(newlyParsedParentQueue.userLimit,
         newlyParsedParentQueue.userLimitFactor,


[04/16] git commit: YARN-1708. Public YARN APIs for creating/updating/deleting reservations. (cherry picked from commit 3f2e3b275bcf29264a112c5d2f4eae289fe7f52c)

Posted by cd...@apache.org.
YARN-1708. Public YARN APIs for creating/updating/deleting reservations.
(cherry picked from commit 3f2e3b275bcf29264a112c5d2f4eae289fe7f52c)


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

Branch: refs/heads/trunk
Commit: c9266df40434fdde05bd3a4c75f7a6bc29db760f
Parents: 1c69503
Author: subru <su...@outlook.com>
Authored: Fri Sep 12 17:16:07 2014 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Oct 3 15:41:39 2014 -0700

----------------------------------------------------------------------
 YARN-1051-CHANGES.txt                           |   2 +
 .../yarn/api/ApplicationClientProtocol.java     | 113 ++++++++++
 .../ReservationDeleteRequest.java               |  72 ++++++
 .../ReservationDeleteResponse.java              |  48 ++++
 .../ReservationSubmissionRequest.java           |  97 ++++++++
 .../ReservationSubmissionResponse.java          |  78 +++++++
 .../ReservationUpdateRequest.java               |  96 ++++++++
 .../ReservationUpdateResponse.java              |  48 ++++
 .../records/ApplicationSubmissionContext.java   |  22 ++
 .../yarn/api/records/ReservationDefinition.java | 137 ++++++++++++
 .../hadoop/yarn/api/records/ReservationId.java  | 147 ++++++++++++
 .../yarn/api/records/ReservationRequest.java    | 222 +++++++++++++++++++
 .../records/ReservationRequestInterpreter.java  | 101 +++++++++
 .../yarn/api/records/ReservationRequests.java   |  94 ++++++++
 .../main/proto/applicationclient_protocol.proto |   3 +
 .../src/main/proto/yarn_protos.proto            |  37 +++-
 .../src/main/proto/yarn_service_protos.proto    |  28 +++
 .../ApplicationClientProtocolPBClientImpl.java  |  58 ++++-
 .../ApplicationClientProtocolPBServiceImpl.java |  61 +++++
 .../impl/pb/ReservationDeleteRequestPBImpl.java | 127 +++++++++++
 .../pb/ReservationDeleteResponsePBImpl.java     |  68 ++++++
 .../pb/ReservationSubmissionRequestPBImpl.java  | 152 +++++++++++++
 .../pb/ReservationSubmissionResponsePBImpl.java | 129 +++++++++++
 .../impl/pb/ReservationUpdateRequestPBImpl.java | 169 ++++++++++++++
 .../pb/ReservationUpdateResponsePBImpl.java     |  68 ++++++
 .../pb/ApplicationSubmissionContextPBImpl.java  |  36 ++-
 .../yarn/api/records/impl/pb/ProtoUtils.java    |  17 ++
 .../impl/pb/ReservationDefinitionPBImpl.java    | 169 ++++++++++++++
 .../records/impl/pb/ReservationIdPBImpl.java    |  75 +++++++
 .../impl/pb/ReservationRequestPBImpl.java       | 152 +++++++++++++
 .../impl/pb/ReservationRequestsPBImpl.java      | 189 ++++++++++++++++
 .../org/apache/hadoop/yarn/util/UTCClock.java   |  39 ++++
 32 files changed, 2848 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/YARN-1051-CHANGES.txt
----------------------------------------------------------------------
diff --git a/YARN-1051-CHANGES.txt b/YARN-1051-CHANGES.txt
index 9fd4b3b..a7c08a0 100644
--- a/YARN-1051-CHANGES.txt
+++ b/YARN-1051-CHANGES.txt
@@ -3,3 +3,5 @@ CapacityScheduler. (Carlo Curino and Subru Krishnan via curino)
 
 YARN-2475. Logic for responding to capacity drops for the 
 ReservationSystem. (Carlo Curino and Subru Krishnan via curino)
+
+YARN-1708. Public YARN APIs for creating/updating/deleting reservations. (subru)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
index e449c1e..863a068 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
@@ -57,6 +57,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesReq
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -67,6 +73,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.Token;
@@ -543,4 +550,110 @@ public interface ApplicationClientProtocol {
   public GetContainersResponse getContainers(GetContainersRequest request)
       throws YarnException, IOException;
 
+  /**
+   * <p>
+   * The interface used by clients to submit a new reservation to the
+   * {@link ResourceManager}.
+   * </p>
+   * 
+   * <p>
+   * The client packages all details of its request in a
+   * {@link ReservationSubmissionRequest} object. This contains information
+   * about the amount of capacity, temporal constraints, and concurrency needs.
+   * Furthermore, the reservation might be composed of multiple stages, with
+   * ordering dependencies among them.
+   * </p>
+   * 
+   * <p>
+   * In order to respond, a new admission control component in the
+   * {@link ResourceManager} performs an analysis of the resources that have
+   * been committed over the period of time the user is requesting, verify that
+   * the user requests can be fulfilled, and that it respect a sharing policy
+   * (e.g., {@link CapacityOverTimePolicy}). Once it has positively determined
+   * that the ReservationSubmissionRequest is satisfiable the
+   * {@link ResourceManager} answers with a
+   * {@link ReservationSubmissionResponse} that include a non-null
+   * {@link ReservationId}. Upon failure to find a valid allocation the response
+   * is an exception with the reason.
+   * 
+   * On application submission the client can use this {@link ReservationId} to
+   * obtain access to the reserved resources.
+   * </p>
+   * 
+   * <p>
+   * The system guarantees that during the time-range specified by the user, the
+   * reservationID will be corresponding to a valid reservation. The amount of
+   * capacity dedicated to such queue can vary overtime, depending of the
+   * allocation that has been determined. But it is guaranteed to satisfy all
+   * the constraint expressed by the user in the
+   * {@link ReservationSubmissionRequest}.
+   * </p>
+   * 
+   * @param request the request to submit a new Reservation
+   * @return response the {@link ReservationId} on accepting the submission
+   * @throws YarnException if the request is invalid or reservation cannot be
+   *           created successfully
+   * @throws IOException
+   * 
+   */
+  @Public
+  @Unstable
+  public ReservationSubmissionResponse submitReservation(
+      ReservationSubmissionRequest request) throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by clients to update an existing Reservation. This is
+   * referred to as a re-negotiation process, in which a user that has
+   * previously submitted a Reservation.
+   * </p>
+   * 
+   * <p>
+   * The allocation is attempted by virtually substituting all previous
+   * allocations related to this Reservation with new ones, that satisfy the new
+   * {@link ReservationUpdateRequest}. Upon success the previous allocation is
+   * substituted by the new one, and on failure (i.e., if the system cannot find
+   * a valid allocation for the updated request), the previous allocation
+   * remains valid.
+   * 
+   * The {@link ReservationId} is not changed, and applications currently
+   * running within this reservation will automatically receive the resources
+   * based on the new allocation.
+   * </p>
+   * 
+   * @param request to update an existing Reservation (the ReservationRequest
+   *          should refer to an existing valid {@link ReservationId})
+   * @return response empty on successfully updating the existing reservation
+   * @throws YarnException if the request is invalid or reservation cannot be
+   *           updated successfully
+   * @throws IOException
+   * 
+   */
+  @Public
+  @Unstable
+  public ReservationUpdateResponse updateReservation(
+      ReservationUpdateRequest request) throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by clients to remove an existing Reservation.
+   * 
+   * Upon deletion of a reservation applications running with this reservation,
+   * are automatically downgraded to normal jobs running without any dedicated
+   * reservation.
+   * </p>
+   * 
+   * @param request to remove an existing Reservation (the ReservationRequest
+   *          should refer to an existing valid {@link ReservationId})
+   * @return response empty on successfully deleting the existing reservation
+   * @throws YarnException if the request is invalid or reservation cannot be
+   *           deleted successfully
+   * @throws IOException
+   * 
+   */
+  @Public
+  @Unstable
+  public ReservationDeleteResponse deleteReservation(
+      ReservationDeleteRequest request) throws YarnException, IOException;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteRequest.java
new file mode 100644
index 0000000..1dd876b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteRequest.java
@@ -0,0 +1,72 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationDeleteRequest} captures the set of requirements the user
+ * has to delete an existing reservation.
+ * 
+ * @see ReservationDefinition
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationDeleteRequest {
+
+  @Public
+  @Unstable
+  public static ReservationDeleteRequest newInstance(ReservationId reservationId) {
+    ReservationDeleteRequest request =
+        Records.newRecord(ReservationDeleteRequest.class);
+    request.setReservationId(reservationId);
+    return request;
+  }
+
+  /**
+   * Get the {@link ReservationId}, that corresponds to a valid resource
+   * allocation in the scheduler (between start and end time of this
+   * reservation)
+   * 
+   * @return the {@link ReservationId} representing the unique id of the
+   *         corresponding reserved resource allocation in the scheduler
+   */
+  @Public
+  @Unstable
+  public abstract ReservationId getReservationId();
+
+  /**
+   * Set the {@link ReservationId}, that correspond to a valid resource
+   * allocation in the scheduler (between start and end time of this
+   * reservation)
+   * 
+   * @param reservationId the {@link ReservationId} representing the the unique
+   *          id of the corresponding reserved resource allocation in the
+   *          scheduler
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationId(ReservationId reservationId);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteResponse.java
new file mode 100644
index 0000000..eedd3a4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteResponse.java
@@ -0,0 +1,48 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationDeleteResponse} contains the answer of the admission
+ * control system in the {@link ResourceManager} to a reservation delete
+ * operation. Currently response is empty if the operation was successful, if
+ * not an exception reporting reason for a failure.
+ * 
+ * @see ReservationDefinition
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationDeleteResponse {
+
+  @Private
+  @Unstable
+  public static ReservationDeleteResponse newInstance() {
+    ReservationDeleteResponse response =
+        Records.newRecord(ReservationDeleteResponse.class);
+    return response;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java
new file mode 100644
index 0000000..e550fe2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java
@@ -0,0 +1,97 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationSubmissionRequest} captures the set of requirements the
+ * user has to create a reservation.
+ * 
+ * @see ReservationDefinition
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationSubmissionRequest {
+
+  @Public
+  @Unstable
+  public static ReservationSubmissionRequest newInstance(
+      ReservationDefinition reservationDefinition, String queueName) {
+    ReservationSubmissionRequest request =
+        Records.newRecord(ReservationSubmissionRequest.class);
+    request.setReservationDefinition(reservationDefinition);
+    request.setQueue(queueName);
+    return request;
+  }
+
+  /**
+   * Get the {@link ReservationDefinition} representing the user constraints for
+   * this reservation
+   * 
+   * @return the reservation definition representing user constraints
+   */
+  @Public
+  @Unstable
+  public abstract ReservationDefinition getReservationDefinition();
+
+  /**
+   * Set the {@link ReservationDefinition} representing the user constraints for
+   * this reservation
+   * 
+   * @param reservationDefinition the reservation request representing the
+   *          reservation
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationDefinition(
+      ReservationDefinition reservationDefinition);
+
+  /**
+   * Get the name of the {@link Plan} that corresponds to the name of the
+   * {@link QueueInfo} in the scheduler to which the reservation will be
+   * submitted to.
+   * 
+   * @return the name of the {@link Plan} that corresponds to the name of the
+   *         {@link QueueInfo} in the scheduler to which the reservation will be
+   *         submitted to
+   */
+  @Public
+  @Unstable
+  public abstract String getQueue();
+
+  /**
+   * Set the name of the {@link Plan} that corresponds to the name of the
+   * {@link QueueInfo} in the scheduler to which the reservation will be
+   * submitted to
+   * 
+   * @param the name of the parent {@link Plan} that corresponds to the name of
+   *          the {@link QueueInfo} in the scheduler to which the reservation
+   *          will be submitted to
+   */
+  @Public
+  @Unstable
+  public abstract void setQueue(String queueName);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java
new file mode 100644
index 0000000..b57ef52
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java
@@ -0,0 +1,78 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.util.Records;
+
+import com.sun.naming.internal.ResourceManager;
+
+/**
+ * {@link ReservationSubmissionResponse} contains the answer of the admission
+ * control system in the {@link ResourceManager} to a reservation create
+ * operation. Response contains a {@link ReservationId} if the operation was
+ * successful, if not an exception reporting reason for a failure.
+ * 
+ * @see ReservationDefinition
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationSubmissionResponse {
+
+  @Private
+  @Unstable
+  public static ReservationSubmissionResponse newInstance(
+      ReservationId reservationId) {
+    ReservationSubmissionResponse response =
+        Records.newRecord(ReservationSubmissionResponse.class);
+    response.setReservationId(reservationId);
+    return response;
+  }
+
+  /**
+   * Get the {@link ReservationId}, that corresponds to a valid resource
+   * allocation in the scheduler (between start and end time of this
+   * reservation)
+   * 
+   * @return the {@link ReservationId} representing the unique id of the
+   *         corresponding reserved resource allocation in the scheduler
+   */
+  @Public
+  @Unstable
+  public abstract ReservationId getReservationId();
+
+  /**
+   * Set the {@link ReservationId}, that correspond to a valid resource
+   * allocation in the scheduler (between start and end time of this
+   * reservation)
+   * 
+   * @param reservationId the {@link ReservationId} representing the the unique
+   *          id of the corresponding reserved resource allocation in the
+   *          scheduler
+   */
+  @Private
+  @Unstable
+  public abstract void setReservationId(ReservationId reservationId);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateRequest.java
new file mode 100644
index 0000000..b5a4b7b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateRequest.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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationUpdateRequest} captures the set of requirements the user
+ * has to update an existing reservation.
+ * 
+ * @see ReservationDefinition
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationUpdateRequest {
+
+  @Public
+  @Unstable
+  public static ReservationUpdateRequest newInstance(
+      ReservationDefinition reservationDefinition, ReservationId reservationId) {
+    ReservationUpdateRequest request =
+        Records.newRecord(ReservationUpdateRequest.class);
+    request.setReservationDefinition(reservationDefinition);
+    request.setReservationId(reservationId);
+    return request;
+  }
+
+  /**
+   * Get the {@link ReservationDefinition} representing the updated user
+   * constraints for this reservation
+   * 
+   * @return the reservation definition representing user constraints
+   */
+  @Public
+  @Unstable
+  public abstract ReservationDefinition getReservationDefinition();
+
+  /**
+   * Set the {@link ReservationDefinition} representing the updated user
+   * constraints for this reservation
+   * 
+   * @param reservationDefinition the reservation request representing the
+   *          reservation
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationDefinition(
+      ReservationDefinition reservationDefinition);
+
+  /**
+   * Get the {@link ReservationId}, that corresponds to a valid resource
+   * allocation in the scheduler (between start and end time of this
+   * reservation)
+   * 
+   * @return the {@link ReservationId} representing the unique id of the
+   *         corresponding reserved resource allocation in the scheduler
+   */
+  @Public
+  @Unstable
+  public abstract ReservationId getReservationId();
+
+  /**
+   * Set the {@link ReservationId}, that correspond to a valid resource
+   * allocation in the scheduler (between start and end time of this
+   * reservation)
+   * 
+   * @param reservationId the {@link ReservationId} representing the the unique
+   *          id of the corresponding reserved resource allocation in the
+   *          scheduler
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationId(ReservationId reservationId);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateResponse.java
new file mode 100644
index 0000000..bfd2d51
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateResponse.java
@@ -0,0 +1,48 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationUpdateResponse} contains the answer of the admission
+ * control system in the {@link ResourceManager} to a reservation update
+ * operation. Currently response is empty if the operation was successful, if
+ * not an exception reporting reason for a failure.
+ * 
+ * @see ReservationDefinition
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationUpdateResponse {
+
+  @Private
+  @Unstable
+  public static ReservationUpdateResponse newInstance() {
+    ReservationUpdateResponse response =
+        Records.newRecord(ReservationUpdateResponse.class);
+    return response;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
index 2202380..cbf1e37 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
@@ -417,4 +417,26 @@ public abstract class ApplicationSubmissionContext {
   @Stable
   public abstract void setLogAggregationContext(
       LogAggregationContext logAggregationContext);
+
+  /**
+   * Get the reservation id, that corresponds to a valid resource allocation in
+   * the scheduler (between start and end time of the corresponding reservation)
+   * 
+   * @return the reservation id representing the unique id of the corresponding
+   *         reserved resource allocation in the scheduler
+   */
+  @Public
+  @Unstable
+  public abstract ReservationId getReservationID();
+
+  /**
+   * Set the reservation id, that correspond to a valid resource allocation in
+   * the scheduler (between start and end time of the corresponding reservation)
+   * 
+   * @param reservationId representing the unique id of the
+   *          corresponding reserved resource allocation in the scheduler
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationID(ReservationId reservationID);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java
new file mode 100644
index 0000000..c94463d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java
@@ -0,0 +1,137 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationDefinition} captures the set of resource and time
+ * constraints the user cares about regarding a reservation.
+ * 
+ * @see ResourceRequest
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationDefinition {
+
+  @Public
+  @Unstable
+  public static ReservationDefinition newInstance(long arrival, long deadline,
+      ReservationRequests reservationRequests, String name) {
+    ReservationDefinition rDefinition =
+        Records.newRecord(ReservationDefinition.class);
+    rDefinition.setArrival(arrival);
+    rDefinition.setDeadline(deadline);
+    rDefinition.setReservationRequests(reservationRequests);
+    rDefinition.setReservationName(name);
+    return rDefinition;
+  }
+
+  /**
+   * Get the arrival time or the earliest time from which the resource(s) can be
+   * allocated. Time expressed as UTC.
+   * 
+   * @return the earliest valid time for this reservation
+   */
+  @Public
+  @Unstable
+  public abstract long getArrival();
+
+  /**
+   * Set the arrival time or the earliest time from which the resource(s) can be
+   * allocated. Time expressed as UTC.
+   * 
+   * @param earliestStartTime the earliest valid time for this reservation
+   */
+  @Public
+  @Unstable
+  public abstract void setArrival(long earliestStartTime);
+
+  /**
+   * Get the deadline or the latest time by when the resource(s) must be
+   * allocated. Time expressed as UTC.
+   * 
+   * @return the deadline or the latest time by when the resource(s) must be
+   *         allocated
+   */
+  @Public
+  @Unstable
+  public abstract long getDeadline();
+
+  /**
+   * Set the deadline or the latest time by when the resource(s) must be
+   * allocated. Time expressed as UTC.
+   * 
+   * @param latestEndTime the deadline or the latest time by when the
+   *          resource(s) should be allocated
+   */
+  @Public
+  @Unstable
+  public abstract void setDeadline(long latestEndTime);
+
+  /**
+   * Get the list of {@link ReservationRequests} representing the resources
+   * required by the application
+   * 
+   * @return the list of {@link ReservationRequests}
+   */
+  @Public
+  @Unstable
+  public abstract ReservationRequests getReservationRequests();
+
+  /**
+   * Set the list of {@link ReservationRequests} representing the resources
+   * required by the application
+   * 
+   * @param resources the list of {@link ReservationRequests}
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationRequests(
+      ReservationRequests reservationRequests);
+
+  /**
+   * Get the name for this reservation. The name need-not be unique, and it is
+   * just a mnemonic for the user (akin to job names). Accepted reservations are
+   * uniquely identified by a system-generated ReservationId.
+   * 
+   * @return string representing the name of the corresponding reserved resource
+   *         allocation in the scheduler
+   */
+  @Public
+  @Evolving
+  public abstract String getReservationName();
+
+  /**
+   * Set the name for this reservation. The name need-not be unique, and it is
+   * just a mnemonic for the user (akin to job names). Accepted reservations are
+   * uniquely identified by a system-generated ReservationId.
+   * 
+   * @param name representing the name of the corresponding reserved resource
+   *          allocation in the scheduler
+   */
+  @Public
+  @Evolving
+  public abstract void setReservationName(String name);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
new file mode 100644
index 0000000..47a8c40
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
@@ -0,0 +1,147 @@
+/**
+ * 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.api.records;
+
+import java.text.NumberFormat;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * {@link ReservationId} represents the <em>globally unique</em> identifier for
+ * a reservation.
+ * </p>
+ * 
+ * <p>
+ * The globally unique nature of the identifier is achieved by using the
+ * <em>cluster timestamp</em> i.e. start-time of the {@link ResourceManager}
+ * along with a monotonically increasing counter for the reservation.
+ * </p>
+ */
+@Public
+@Unstable
+public abstract class ReservationId implements Comparable<ReservationId> {
+
+  @Private
+  @Unstable
+  public static final String reserveIdStrPrefix = "reservation_";
+  protected long clusterTimestamp;
+  protected long id;
+
+  @Private
+  @Unstable
+  public static ReservationId newInstance(long clusterTimestamp, long id) {
+    ReservationId reservationId = Records.newRecord(ReservationId.class);
+    reservationId.setClusterTimestamp(clusterTimestamp);
+    reservationId.setId(id);
+    reservationId.build();
+    return reservationId;
+  }
+
+  /**
+   * Get the long identifier of the {@link ReservationId} which is unique for
+   * all Reservations started by a particular instance of the
+   * {@link ResourceManager}.
+   * 
+   * @return long identifier of the {@link ReservationId}
+   */
+  @Public
+  @Unstable
+  public abstract long getId();
+
+  @Private
+  @Unstable
+  protected abstract void setId(long id);
+
+  /**
+   * Get the <em>start time</em> of the {@link ResourceManager} which is used to
+   * generate globally unique {@link ReservationId}.
+   * 
+   * @return <em>start time</em> of the {@link ResourceManager}
+   */
+  @Public
+  @Unstable
+  public abstract long getClusterTimestamp();
+
+  @Private
+  @Unstable
+  protected abstract void setClusterTimestamp(long clusterTimestamp);
+
+  protected abstract void build();
+
+  static final ThreadLocal<NumberFormat> reservIdFormat =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(4);
+          return fmt;
+        }
+      };
+
+  @Override
+  public int compareTo(ReservationId other) {
+    if (this.getClusterTimestamp() - other.getClusterTimestamp() == 0) {
+      return getId() > getId() ? 1 : getId() < getId() ? -1 : 0;
+    } else {
+      return this.getClusterTimestamp() > other.getClusterTimestamp() ? 1
+          : this.getClusterTimestamp() < other.getClusterTimestamp() ? -1 : 0;
+    }
+  }
+
+  @Override
+  public String toString() {
+    return reserveIdStrPrefix + this.getClusterTimestamp() + "_"
+        + reservIdFormat.get().format(getId());
+  }
+
+  @Override
+  public int hashCode() {
+    // generated by eclipse
+    final int prime = 31;
+    int result = 1;
+    result =
+        prime * result
+            + (int) (getClusterTimestamp() ^ (getClusterTimestamp() >>> 32));
+    result = prime * result + (int) (getId() ^ (getId() >>> 32));
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    // generated by eclipse
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    ReservationId other = (ReservationId) obj;
+    if (getClusterTimestamp() != other.getClusterTimestamp())
+      return false;
+    if (getId() != other.getId())
+      return false;
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
new file mode 100644
index 0000000..ded6d06
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
@@ -0,0 +1,222 @@
+/**
+ * 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.api.records;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * {@link ReservationRequest} represents the request made by an application to
+ * the {@link ResourceManager} to reserve {@link Resource}s.
+ * </p>
+ * 
+ * <p>
+ * It includes:
+ * <ul>
+ * <li>{@link Resource} required for each request.</li>
+ * <li>
+ * Number of containers, of above specifications, which are required by the
+ * application.</li>
+ * <li>
+ * Concurrency that indicates the gang size of the request.</li>
+ * </ul>
+ * </p>
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationRequest implements
+    Comparable<ReservationRequest> {
+
+  @Public
+  @Unstable
+  public static ReservationRequest newInstance(Resource capability,
+      int numContainers) {
+    return newInstance(capability, numContainers, 1, -1);
+  }
+
+  @Public
+  @Unstable
+  public static ReservationRequest newInstance(Resource capability,
+      int numContainers, int concurrency, long duration) {
+    ReservationRequest request = Records.newRecord(ReservationRequest.class);
+    request.setCapability(capability);
+    request.setNumContainers(numContainers);
+    request.setConcurrency(concurrency);
+    request.setDuration(duration);
+    return request;
+  }
+
+  @Public
+  @Unstable
+  public static class ReservationRequestComparator implements
+      java.util.Comparator<ReservationRequest>, Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public int compare(ReservationRequest r1, ReservationRequest r2) {
+      // Compare numContainers, concurrency and capability
+      int ret = r1.getNumContainers() - r2.getNumContainers();
+      if (ret == 0) {
+        ret = r1.getConcurrency() - r2.getConcurrency();
+      }
+      if (ret == 0) {
+        ret = r1.getCapability().compareTo(r2.getCapability());
+      }
+      return ret;
+    }
+  }
+
+  /**
+   * Get the {@link Resource} capability of the request.
+   * 
+   * @return {@link Resource} capability of the request
+   */
+  @Public
+  @Unstable
+  public abstract Resource getCapability();
+
+  /**
+   * Set the {@link Resource} capability of the request
+   * 
+   * @param capability {@link Resource} capability of the request
+   */
+  @Public
+  @Unstable
+  public abstract void setCapability(Resource capability);
+
+  /**
+   * Get the number of containers required with the given specifications.
+   * 
+   * @return number of containers required with the given specifications
+   */
+  @Public
+  @Unstable
+  public abstract int getNumContainers();
+
+  /**
+   * Set the number of containers required with the given specifications
+   * 
+   * @param numContainers number of containers required with the given
+   *          specifications
+   */
+  @Public
+  @Unstable
+  public abstract void setNumContainers(int numContainers);
+
+  /**
+   * Get the number of containers that need to be scheduled concurrently. The
+   * default value of 1 would fall back to the current non concurrency
+   * constraints on the scheduling behavior.
+   * 
+   * @return the number of containers to be concurrently scheduled
+   */
+  @Public
+  @Unstable
+  public abstract int getConcurrency();
+
+  /**
+   * Set the number of containers that need to be scheduled concurrently. The
+   * default value of 1 would fall back to the current non concurrency
+   * constraints on the scheduling behavior.
+   * 
+   * @param numContainers the number of containers to be concurrently scheduled
+   */
+  @Public
+  @Unstable
+  public abstract void setConcurrency(int numContainers);
+
+  /**
+   * Get the duration in milliseconds for which the resource is required. A
+   * default value of -1, indicates an unspecified lease duration, and fallback
+   * to current behavior.
+   * 
+   * @return the duration in milliseconds for which the resource is required
+   */
+  @Public
+  @Unstable
+  public abstract long getDuration();
+
+  /**
+   * Set the duration in milliseconds for which the resource is required.
+   * 
+   * @param duration the duration in milliseconds for which the resource is
+   *          required
+   */
+  @Public
+  @Unstable
+  public abstract void setDuration(long duration);
+
+  @Override
+  public int hashCode() {
+    final int prime = 2153;
+    int result = 2459;
+    Resource capability = getCapability();
+    result =
+        prime * result + ((capability == null) ? 0 : capability.hashCode());
+    result = prime * result + getNumContainers();
+    result = prime * result + getConcurrency();
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    ReservationRequest other = (ReservationRequest) obj;
+    Resource capability = getCapability();
+    if (capability == null) {
+      if (other.getCapability() != null)
+        return false;
+    } else if (!capability.equals(other.getCapability()))
+      return false;
+    if (getNumContainers() != other.getNumContainers())
+      return false;
+    if (getConcurrency() != other.getConcurrency())
+      return false;
+    return true;
+  }
+
+  @Override
+  public int compareTo(ReservationRequest other) {
+    int numContainersComparison =
+        this.getNumContainers() - other.getNumContainers();
+    if (numContainersComparison == 0) {
+      int concurrencyComparison =
+          this.getConcurrency() - other.getConcurrency();
+      if (concurrencyComparison == 0) {
+        return this.getCapability().compareTo(other.getCapability());
+      } else {
+        return concurrencyComparison;
+      }
+    } else {
+      return numContainersComparison;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
new file mode 100644
index 0000000..4621e0d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
@@ -0,0 +1,101 @@
+package org.apache.hadoop.yarn.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+/**
+ * Enumeration of various types of dependencies among multiple
+ * {@link ReservationRequests} within one {@link ReservationDefinition} (from
+ * least constraining to most constraining).
+ */
+@Public
+@Evolving
+public enum ReservationRequestInterpreter {
+  /**
+   * Requires that exactly ONE among the {@link ReservationRequest} submitted as
+   * of a {@link ReservationDefinition} is satisfied to satisfy the overall
+   * {@link ReservationDefinition}.
+   * 
+   * WHEN TO USE THIS: This is useful when the user have multiple equivalent
+   * ways to run an application, and wants to expose to the ReservationAgent
+   * such flexibility. For example an application could use one <32GB,16core>
+   * container for 10min, or 16 <2GB,1core> containers for 15min, the
+   * ReservationAgent will decide which one of the two it is best for the system
+   * to place.
+   * 
+   */
+  R_ANY,
+
+  /**
+   * Requires that ALL of the {@link ReservationRequest} submitted as part of a
+   * {@link ReservationDefinition} are satisfied for the overall
+   * {@link ReservationDefinition} to be satisfied. No constraints are imposed
+   * on the temporal ordering of the allocation used to satisfy the
+   * ResourceRequeusts.
+   * 
+   * WHEN TO USE THIS: This is useful to capture a scenario in which the user
+   * cares for multiple ReservationDefinition to be all accepted, or none. For
+   * example, a user might want a reservation R1: with 10 x <8GB,4core> for
+   * 10min, and a reservation R2: with 2 <1GB,1core> for 1h, and only if both
+   * are satisfied the workflow run in this reservation succeeds. The key
+   * differentiator from ALL and ORDER, ORDER_NO_GAP, is that ALL imposes no
+   * restrictions on the relative allocations used to place R1 and R2 above.
+   * 
+   */
+  R_ALL,
+
+  /**
+   * Requires that ALL of the {@link ReservationRequest} submitted as part of a
+   * {@link ReservationDefinition} are satisfied for the overall
+   * {@link ReservationDefinition} to be satisfied. Moreover, it imposes a
+   * strict temporal ordering on the allocation used to satisfy the
+   * {@link ReservationRequest}s. The allocations satisfying the
+   * {@link ReservationRequest} in position k must strictly precede the
+   * allocations for the {@link ReservationRequest} at position k+1. No
+   * constraints are imposed on temporal gaps between subsequent allocations
+   * (the last instant of the previous allocation can be an arbitrary long
+   * period of time before the first instant of the subsequent allocation).
+   * 
+   * WHEN TO USE THIS: Like ALL this requires all ReservationDefinitions to be
+   * placed, but it also imposes a time ordering on the allocations used. This
+   * is important if the ReservationDefinition(s) are used to describe a
+   * workflow with inherent inter-stage dependencies. For example, a first job
+   * runs in a ReservaitonDefinition R1 (10 x <1GB,1core> for 20min), and its
+   * output is consumed by a second job described by a ReservationDefinition R2
+   * (5 x <1GB,1core>) for 50min). R2 allocation cannot overlap R1, as R2 models
+   * a job depending on the output of the job modeled by R1.
+   */
+  R_ORDER,
+
+  /**
+   * Requires that ALL of the {@link ReservationRequest} submitted as part of a
+   * {@link ReservationDefinition} are satisfied for the overall
+   * {@link ReservationDefinition} to be satisfied. Moreover, it imposes a
+   * strict temporal ordering on the allocation used to satisfy the
+   * {@link ResourceRequeust}s. It imposes a strict temporal ordering on the
+   * allocation used to satisfy the {@link ReservationRequest}s. The allocations
+   * satisfying the {@link ReservationRequest} in position k must strictly
+   * precede the allocations for the {@link ReservationRequest} at position k+1.
+   * Moreover it imposes a "zero-size gap" between subsequent allocations, i.e.,
+   * the last instant in time of the allocations associated with the
+   * {@link ReservationRequest} at position k must be exactly preceding the
+   * first instant in time of the {@link ReservationRequest} at position k+1.
+   * Time ranges are interpreted as [a,b) inclusive left, exclusive right.
+   * 
+   * WHEN TO USE THIS: This is a stricter version of R_ORDER, which allows no
+   * gaps between the allocations that satisfy R1 and R2. The use of this is
+   * twofold: 1) prevent long gaps between subsequent stages that produce very
+   * large intermediate output (e.g., the output of R1 is too large to be kept
+   * around for long before the job running in R2 consumes it, and disposes of
+   * it), 2) if the job being modeled has a time-varying resource need, one can
+   * combine multiple ResourceDefinition each approximating a portion of the job
+   * execution (think of using multiple rectangular bounding boxes to described
+   * an arbitrarily shaped area). By asking for no-gaps we guarantee
+   * "continuity" of resources given to this job. This still allow for some
+   * flexibility, as the entire "train" of allocations can be moved rigidly back
+   * or forth within the start-deadline time range (if there is slack).
+   * 
+   */
+  R_ORDER_NO_GAP
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequests.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequests.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequests.java
new file mode 100644
index 0000000..8cc3fbc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequests.java
@@ -0,0 +1,94 @@
+/**
+ * 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.api.records;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationRequests} captures the set of resource and constraints the
+ * user cares about regarding a reservation.
+ * 
+ * @see ReservationRequest
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationRequests {
+
+  @Public
+  @Unstable
+  public static ReservationRequests newInstance(
+      List<ReservationRequest> reservationResources,
+      ReservationRequestInterpreter type) {
+    ReservationRequests reservationRequests =
+        Records.newRecord(ReservationRequests.class);
+    reservationRequests.setReservationResources(reservationResources);
+    reservationRequests.setInterpreter(type);
+    return reservationRequests;
+  }
+
+  /**
+   * Get the list of {@link ReservationRequest} representing the resources
+   * required by the application
+   * 
+   * @return the list of {@link ReservationRequest}
+   */
+  @Public
+  @Unstable
+  public abstract List<ReservationRequest> getReservationResources();
+
+  /**
+   * Set the list of {@link ReservationRequest} representing the resources
+   * required by the application
+   * 
+   * @param resources the list of {@link ReservationRequest}
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationResources(
+      List<ReservationRequest> reservationResources);
+
+  /**
+   * Get the {@link ReservationRequestInterpreter}, representing how the list of
+   * resources should be allocated, this captures temporal ordering and other
+   * constraints.
+   * 
+   * @return the list of {@link ReservationRequestInterpreter}
+   */
+  @Public
+  @Unstable
+  public abstract ReservationRequestInterpreter getInterpreter();
+
+  /**
+   * Set the {@link ReservationRequestInterpreter}, representing how the list of
+   * resources should be allocated, this captures temporal ordering and other
+   * constraints.
+   * 
+   * @param interpreter the {@link ReservationRequestInterpreter} for this
+   *          reservation
+   */
+  @Public
+  @Unstable
+  public abstract void setInterpreter(ReservationRequestInterpreter interpreter);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
index 2d8cae6..3346aef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
@@ -49,5 +49,8 @@ service ApplicationClientProtocolService {
   rpc getApplicationAttempts (GetApplicationAttemptsRequestProto) returns (GetApplicationAttemptsResponseProto);
   rpc getContainerReport (GetContainerReportRequestProto) returns (GetContainerReportResponseProto);
   rpc getContainers (GetContainersRequestProto) returns (GetContainersResponseProto);
+  rpc submitReservation (ReservationSubmissionRequestProto) returns (ReservationSubmissionResponseProto);
+  rpc updateReservation (ReservationUpdateRequestProto) returns (ReservationUpdateResponseProto);
+  rpc deleteReservation (ReservationDeleteRequestProto) returns (ReservationDeleteResponseProto);
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index b368746..c645719 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -202,7 +202,6 @@ message ApplicationAttemptReportProto {
   optional string diagnostics = 5 [default = "N/A"];
   optional YarnApplicationAttemptStateProto yarn_application_attempt_state = 6;
   optional ContainerIdProto am_container_id = 7;
-  optional string original_tracking_url = 8;
 }
 
 enum NodeStateProto {
@@ -293,6 +292,7 @@ message ApplicationSubmissionContextProto {
   repeated string applicationTags = 12;
   optional int64 attempt_failures_validity_interval = 13 [default = -1];
   optional LogAggregationContextProto log_aggregation_context = 14;
+  optional ReservationIdProto reservation_id = 15;
 }
 
 message LogAggregationContextProto {
@@ -341,6 +341,41 @@ message QueueUserACLInfoProto {
 }
 
 ////////////////////////////////////////////////////////////////////////
+////// From reservation_protocol /////////////////////////////////////
+////////////////////////////////////////////////////////////////////////
+
+message ReservationIdProto {
+  optional int64 id = 1;
+  optional int64 cluster_timestamp = 2;
+}
+
+message ReservationRequestProto {
+  optional ResourceProto capability = 1;
+  optional int32 num_containers = 2 [default = 1];
+  optional int32 concurrency = 3 [default = 1];
+  optional int64 duration = 4 [default = -1];
+}
+
+message ReservationRequestsProto {
+  repeated ReservationRequestProto reservation_resources = 1;
+  optional ReservationRequestInterpreterProto interpreter = 2 [default = R_ALL];
+}
+
+message ReservationDefinitionProto {
+  optional ReservationRequestsProto reservation_requests = 1;
+  optional int64 arrival = 2;
+  optional int64 deadline = 3;
+  optional string reservation_name = 4;
+}
+
+enum ReservationRequestInterpreterProto {
+  R_ANY = 0;
+  R_ALL = 1;
+  R_ORDER = 2;
+  R_ORDER_NO_GAP = 3;
+ }
+
+////////////////////////////////////////////////////////////////////////
 ////// From container_manager //////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index 4203744..2585f78 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -292,3 +292,31 @@ message GetContainersRequestProto {
 message GetContainersResponseProto {
   repeated ContainerReportProto containers = 1;
 }
+
+//////////////////////////////////////////////////////
+//  reservation_protocol
+//////////////////////////////////////////////////////
+
+message ReservationSubmissionRequestProto {
+  optional string queue = 1;
+  optional ReservationDefinitionProto reservation_definition = 2;
+}
+
+message ReservationSubmissionResponseProto {
+    optional ReservationIdProto reservation_id = 1;
+}
+
+message ReservationUpdateRequestProto {
+  optional ReservationDefinitionProto reservation_definition = 1;
+  optional ReservationIdProto reservation_id = 2;
+}
+
+message ReservationUpdateResponseProto {   
+}
+
+message ReservationDeleteRequestProto {
+  optional ReservationIdProto reservation_id = 1;
+}
+
+message ReservationDeleteResponseProto {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
index 5794da0..b4b9f43 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
@@ -63,6 +63,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesReq
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenRequestPBImpl;
@@ -97,6 +103,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQ
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQueuesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationDeleteRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationDeleteResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationSubmissionRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationSubmissionResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationUpdateRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationUpdateResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -110,13 +122,15 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerReportRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainersRequestProto;
 
-
 import com.google.protobuf.ServiceException;
 
 @Private
@@ -391,4 +405,46 @@ public class ApplicationClientProtocolPBClientImpl implements ApplicationClientP
     }
   }
 
+  @Override
+  public ReservationSubmissionResponse submitReservation(ReservationSubmissionRequest request)
+      throws YarnException, IOException {
+    ReservationSubmissionRequestProto requestProto =
+        ((ReservationSubmissionRequestPBImpl) request).getProto();
+    try {
+      return new ReservationSubmissionResponsePBImpl(proxy.submitReservation(null,
+          requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
+
+  @Override
+  public ReservationUpdateResponse updateReservation(ReservationUpdateRequest request)
+      throws YarnException, IOException {
+    ReservationUpdateRequestProto requestProto =
+        ((ReservationUpdateRequestPBImpl) request).getProto();
+    try {
+      return new ReservationUpdateResponsePBImpl(proxy.updateReservation(null,
+          requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
+
+  @Override
+  public ReservationDeleteResponse deleteReservation(ReservationDeleteRequest request)
+      throws YarnException, IOException {
+    ReservationDeleteRequestProto requestProto =
+        ((ReservationDeleteRequestPBImpl) request).getProto();
+    try {
+      return new ReservationDeleteResponsePBImpl(proxy.deleteReservation(null,
+          requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
index 1d9bedd..f73202d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
@@ -45,6 +45,9 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenResponsePBImpl;
@@ -78,6 +81,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQ
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQueuesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationDeleteRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationDeleteResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationSubmissionRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationSubmissionResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationUpdateRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationUpdateResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -99,6 +108,12 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProt
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportRequestProto;
@@ -383,4 +398,50 @@ public class ApplicationClientProtocolPBServiceImpl implements ApplicationClient
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public ReservationSubmissionResponseProto submitReservation(RpcController controller,
+      ReservationSubmissionRequestProto requestProto) throws ServiceException {
+    ReservationSubmissionRequestPBImpl request =
+        new ReservationSubmissionRequestPBImpl(requestProto);
+    try {
+      ReservationSubmissionResponse response = real.submitReservation(request);
+      return ((ReservationSubmissionResponsePBImpl) response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ReservationUpdateResponseProto updateReservation(RpcController controller,
+      ReservationUpdateRequestProto requestProto) throws ServiceException {
+    ReservationUpdateRequestPBImpl request =
+        new ReservationUpdateRequestPBImpl(requestProto);
+    try {
+      ReservationUpdateResponse response = real.updateReservation(request);
+      return ((ReservationUpdateResponsePBImpl) response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ReservationDeleteResponseProto deleteReservation(RpcController controller,
+      ReservationDeleteRequestProto requestProto) throws ServiceException {
+    ReservationDeleteRequestPBImpl request =
+        new ReservationDeleteRequestPBImpl(requestProto);
+    try {
+      ReservationDeleteResponse response = real.deleteReservation(request);
+      return ((ReservationDeleteResponsePBImpl) response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationDeleteRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationDeleteRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationDeleteRequestPBImpl.java
new file mode 100644
index 0000000..c235dce
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationDeleteRequestPBImpl.java
@@ -0,0 +1,127 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationIdPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationIdProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteRequestProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+public class ReservationDeleteRequestPBImpl extends ReservationDeleteRequest {
+
+  ReservationDeleteRequestProto proto = ReservationDeleteRequestProto
+      .getDefaultInstance();
+  ReservationDeleteRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ReservationId reservationId;
+
+  public ReservationDeleteRequestPBImpl() {
+    builder = ReservationDeleteRequestProto.newBuilder();
+  }
+
+  public ReservationDeleteRequestPBImpl(ReservationDeleteRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationDeleteRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationId != null) {
+      builder.setReservationId(convertToProtoFormat(this.reservationId));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationDeleteRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public ReservationId getReservationId() {
+    ReservationDeleteRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationId != null) {
+      return reservationId;
+    }
+    if (!p.hasReservationId()) {
+      return null;
+    }
+    reservationId = convertFromProtoFormat(p.getReservationId());
+    return reservationId;
+  }
+
+  @Override
+  public void setReservationId(ReservationId reservationId) {
+    maybeInitBuilder();
+    if (reservationId == null) {
+      builder.clearReservationId();
+      return;
+    }
+    this.reservationId = reservationId;
+  }
+
+  private ReservationIdPBImpl convertFromProtoFormat(ReservationIdProto p) {
+    return new ReservationIdPBImpl(p);
+  }
+
+  private ReservationIdProto convertToProtoFormat(ReservationId t) {
+    return ((ReservationIdPBImpl) t).getProto();
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9266df4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationDeleteResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationDeleteResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationDeleteResponsePBImpl.java
new file mode 100644
index 0000000..a249ab8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationDeleteResponsePBImpl.java
@@ -0,0 +1,68 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteResponseProto;
+
+import com.google.protobuf.TextFormat;
+
+public class ReservationDeleteResponsePBImpl extends ReservationDeleteResponse {
+
+  ReservationDeleteResponseProto proto = ReservationDeleteResponseProto
+      .getDefaultInstance();
+  ReservationDeleteResponseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  public ReservationDeleteResponsePBImpl() {
+    builder = ReservationDeleteResponseProto.newBuilder();
+  }
+
+  public ReservationDeleteResponsePBImpl(ReservationDeleteResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationDeleteResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}


[07/16] git commit: YARN-1710. Logic to find allocations within a Plan that satisfy user ReservationRequest(s). Contributed by Carlo Curino and Subru Krishnan. (cherry picked from commit aef7928899b37262773f3dc117157bb746bf8918)

Posted by cd...@apache.org.
YARN-1710. Logic to find allocations within a Plan that satisfy user ReservationRequest(s). Contributed by Carlo Curino and Subru Krishnan.
(cherry picked from commit aef7928899b37262773f3dc117157bb746bf8918)


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

Branch: refs/heads/trunk
Commit: f66ffcf832235e0da0bb050fff08e248b547c360
Parents: cf4b342
Author: carlo curino <Carlo Curino>
Authored: Mon Sep 15 16:56:28 2014 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Oct 3 15:41:57 2014 -0700

----------------------------------------------------------------------
 YARN-1051-CHANGES.txt                           |  12 +-
 .../reservation/GreedyReservationAgent.java     | 367 ++++++++++++
 .../reservation/ReservationAgent.java           |  55 ++
 .../exceptions/ContractValidationException.java |  12 +
 .../reservation/TestGreedyReservationAgent.java | 588 +++++++++++++++++++
 5 files changed, 1031 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66ffcf8/YARN-1051-CHANGES.txt
----------------------------------------------------------------------
diff --git a/YARN-1051-CHANGES.txt b/YARN-1051-CHANGES.txt
index 410d974..deece7c 100644
--- a/YARN-1051-CHANGES.txt
+++ b/YARN-1051-CHANGES.txt
@@ -4,7 +4,13 @@ CapacityScheduler. (Carlo Curino and Subru Krishnan via curino)
 YARN-2475. Logic for responding to capacity drops for the 
 ReservationSystem. (Carlo Curino and Subru Krishnan via curino)
 
-YARN-1708. Public YARN APIs for creating/updating/deleting reservations. (subru)
+YARN-1708. Public YARN APIs for creating/updating/deleting 
+reservations. (Carlo Curino and Subru Krishnan via subru)
 
-YARN-1709. In-memory data structures used to track resources over time to
-enable reservations. (subru)
+YARN-1709. In-memory data structures used to track resources over
+time to enable reservations. (Carlo Curino and Subru Krishnan via 
+subru)
+
+YARN-1710. Logic to find allocations within a Plan that satisfy 
+user ReservationRequest(s). (Carlo Curino and Subru Krishnan via 
+curino) 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66ffcf8/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
new file mode 100644
index 0000000..3214f93
--- /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/GreedyReservationAgent.java
@@ -0,0 +1,367 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+
+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.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.ContractValidationException;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This Agent employs a simple greedy placement strategy, placing the various
+ * stages of a {@link ReservationRequest} from the deadline moving backward
+ * towards the arrival. This allows jobs with earlier deadline to be scheduled
+ * greedily as well. Combined with an opportunistic anticipation of work if the
+ * cluster is not fully utilized also seems to provide good latency for
+ * best-effort jobs (i.e., jobs running without a reservation).
+ * 
+ * This agent does not account for locality and only consider container
+ * granularity for validation purposes (i.e., you can't exceed max-container
+ * size).
+ */
+public class GreedyReservationAgent implements ReservationAgent {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(GreedyReservationAgent.class);
+
+  @Override
+  public boolean createReservation(ReservationId reservationId, String user,
+      Plan plan, ReservationDefinition contract) throws PlanningException {
+    return computeAllocation(reservationId, user, plan, contract, null);
+  }
+
+  @Override
+  public boolean updateReservation(ReservationId reservationId, String user,
+      Plan plan, ReservationDefinition contract) throws PlanningException {
+    return computeAllocation(reservationId, user, plan, contract,
+        plan.getReservationById(reservationId));
+  }
+
+  @Override
+  public boolean deleteReservation(ReservationId reservationId, String user,
+      Plan plan) throws PlanningException {
+    return plan.deleteReservation(reservationId);
+  }
+
+  private boolean computeAllocation(ReservationId reservationId, String user,
+      Plan plan, ReservationDefinition contract,
+      ReservationAllocation oldReservation) throws PlanningException,
+      ContractValidationException {
+    LOG.info("placing the following ReservationRequest: " + contract);
+
+    Resource totalCapacity = plan.getTotalCapacity();
+
+    // Here we can addd logic to adjust the ResourceDefinition to account for
+    // system "imperfections" (e.g., scheduling delays for large containers).
+
+    // Align with plan step conservatively (i.e., ceil arrival, and floor
+    // deadline)
+    long earliestStart = contract.getArrival();
+    long step = plan.getStep();
+    if (earliestStart % step != 0) {
+      earliestStart = earliestStart + (step - (earliestStart % step));
+    }
+    long deadline =
+        contract.getDeadline() - contract.getDeadline() % plan.getStep();
+
+    // setup temporary variables to handle time-relations between stages and
+    // intermediate answers
+    long curDeadline = deadline;
+    long oldDeadline = -1;
+
+    Map<ReservationInterval, ReservationRequest> allocations =
+        new HashMap<ReservationInterval, ReservationRequest>();
+    RLESparseResourceAllocation tempAssigned =
+        new RLESparseResourceAllocation(plan.getResourceCalculator(),
+            plan.getMinimumAllocation());
+
+    List<ReservationRequest> stages = contract.getReservationRequests()
+        .getReservationResources();
+    ReservationRequestInterpreter type = contract.getReservationRequests()
+        .getInterpreter();
+
+    // Iterate the stages in backward from deadline
+    for (ListIterator<ReservationRequest> li = 
+        stages.listIterator(stages.size()); li.hasPrevious();) {
+
+      ReservationRequest currentReservationStage = li.previous();
+
+      // validate the RR respect basic constraints
+      validateInput(plan, currentReservationStage, totalCapacity);
+
+      // run allocation for a single stage
+      Map<ReservationInterval, ReservationRequest> curAlloc =
+          placeSingleStage(plan, tempAssigned, currentReservationStage,
+              earliestStart, curDeadline, oldReservation, totalCapacity);
+
+      if (curAlloc == null) {
+        // if we did not find an allocation for the currentReservationStage
+        // return null, unless the ReservationDefinition we are placing is of
+        // type ANY
+        if (type != ReservationRequestInterpreter.R_ANY) {
+          throw new PlanningException("The GreedyAgent"
+              + " couldn't find a valid allocation for your request");
+        } else {
+          continue;
+        }
+      } else {
+
+        // if we did find an allocation add it to the set of allocations
+        allocations.putAll(curAlloc);
+
+        // if this request is of type ANY we are done searching (greedy)
+        // and can return the current allocation (break-out of the search)
+        if (type == ReservationRequestInterpreter.R_ANY) {
+          break;
+        }
+
+        // if the request is of ORDER or ORDER_NO_GAP we constraint the next
+        // round of allocation to precede the current allocation, by setting
+        // curDeadline
+        if (type == ReservationRequestInterpreter.R_ORDER
+            || type == ReservationRequestInterpreter.R_ORDER_NO_GAP) {
+          curDeadline = findEarliestTime(curAlloc.keySet());
+
+          // for ORDER_NO_GAP verify that the allocation found so far has no
+          // gap, return null otherwise (the greedy procedure failed to find a
+          // no-gap
+          // allocation)
+          if (type == ReservationRequestInterpreter.R_ORDER_NO_GAP
+              && oldDeadline > 0) {
+            if (oldDeadline - findLatestTime(curAlloc.keySet()) > plan
+                .getStep()) {
+              throw new PlanningException("The GreedyAgent"
+                  + " couldn't find a valid allocation for your request");
+            }
+          }
+          // keep the variable oldDeadline pointing to the last deadline we
+          // found
+          oldDeadline = curDeadline;
+        }
+      }
+    }
+
+    // / If we got here is because we failed to find an allocation for the
+    // ReservationDefinition give-up and report failure to the user
+    if (allocations.isEmpty()) {
+      throw new PlanningException("The GreedyAgent"
+          + " couldn't find a valid allocation for your request");
+    }
+
+    // create reservation with above allocations if not null/empty
+
+    ReservationRequest ZERO_RES =
+        ReservationRequest.newInstance(Resource.newInstance(0, 0), 0);
+
+    long firstStartTime = findEarliestTime(allocations.keySet());
+    
+    // add zero-padding from arrival up to the first non-null allocation
+    // to guarantee that the reservation exists starting at arrival
+    if (firstStartTime > earliestStart) {
+      allocations.put(new ReservationInterval(earliestStart,
+          firstStartTime), ZERO_RES);
+      firstStartTime = earliestStart;
+      // consider to add trailing zeros at the end for simmetry
+    }
+
+    // Actually add/update the reservation in the plan.
+    // This is subject to validation as other agents might be placing
+    // in parallel and there might be sharing policies the agent is not
+    // aware off.
+    ReservationAllocation capReservation =
+        new InMemoryReservationAllocation(reservationId, contract, user,
+            plan.getQueueName(), firstStartTime,
+            findLatestTime(allocations.keySet()), allocations,
+            plan.getResourceCalculator(), plan.getMinimumAllocation());
+    if (oldReservation != null) {
+      return plan.updateReservation(capReservation);
+    } else {
+      return plan.addReservation(capReservation);
+    }
+  }
+
+  private void validateInput(Plan plan, ReservationRequest rr,
+      Resource totalCapacity) throws ContractValidationException {
+
+    if (rr.getConcurrency() < 1) {
+      throw new ContractValidationException("Gang Size should be >= 1");
+    }
+
+    if (rr.getNumContainers() <= 0) {
+      throw new ContractValidationException("Num containers should be >= 0");
+    }
+
+    // check that gangSize and numContainers are compatible
+    if (rr.getNumContainers() % rr.getConcurrency() != 0) {
+      throw new ContractValidationException(
+          "Parallelism must be an exact multiple of gang size");
+    }
+
+    // check that the largest container request does not exceed
+    // the cluster-wide limit for container sizes
+    if (Resources.greaterThan(plan.getResourceCalculator(), totalCapacity,
+        rr.getCapability(), plan.getMaximumAllocation())) {
+      throw new ContractValidationException("Individual"
+          + " capability requests should not exceed cluster's maxAlloc");
+    }
+  }
+
+  /**
+   * This method actually perform the placement of an atomic stage of the
+   * reservation. The key idea is to traverse the plan backward for a
+   * "lease-duration" worth of time, and compute what is the maximum multiple of
+   * our concurrency (gang) parameter we can fit. We do this and move towards
+   * previous instant in time until the time-window is exhausted or we placed
+   * all the user request.
+   */
+  private Map<ReservationInterval, ReservationRequest> placeSingleStage(
+      Plan plan, RLESparseResourceAllocation tempAssigned,
+      ReservationRequest rr, long earliestStart, long curDeadline,
+      ReservationAllocation oldResAllocation, final Resource totalCapacity) {
+
+    Map<ReservationInterval, ReservationRequest> allocationRequests =
+        new HashMap<ReservationInterval, ReservationRequest>();
+
+    // compute the gang as a resource and get the duration
+    Resource gang = Resources.multiply(rr.getCapability(), rr.getConcurrency());
+    long dur = rr.getDuration();
+    long step = plan.getStep();
+
+    // ceil the duration to the next multiple of the plan step
+    if (dur % step != 0) {
+      dur += (step - (dur % step));
+    }
+
+    // we know for sure that this division has no remainder (part of contract
+    // with user, validate before
+    int gangsToPlace = rr.getNumContainers() / rr.getConcurrency();
+
+    int maxGang = 0;
+
+    // loop trying to place until we are done, or we are considering
+    // an invalid range of times
+    while (gangsToPlace > 0 && curDeadline - dur >= earliestStart) {
+
+      // as we run along we remember how many gangs we can fit, and what
+      // was the most constraining moment in time (we will restart just
+      // after that to place the next batch)
+      maxGang = gangsToPlace;
+      long minPoint = curDeadline;
+      int curMaxGang = maxGang;
+
+      // start placing at deadline (excluded due to [,) interval semantics and
+      // move backward
+      for (long t = curDeadline - plan.getStep(); t >= curDeadline - dur
+          && maxGang > 0; t = t - plan.getStep()) {
+
+        // As we run along we will logically remove the previous allocation for
+        // this reservation
+        // if one existed
+        Resource oldResCap = Resource.newInstance(0, 0);
+        if (oldResAllocation != null) {
+          oldResCap = oldResAllocation.getResourcesAtTime(t);
+        }
+
+        // compute net available resources
+        Resource netAvailableRes = Resources.clone(totalCapacity);
+        Resources.addTo(netAvailableRes, oldResCap);
+        Resources.subtractFrom(netAvailableRes,
+            plan.getTotalCommittedResources(t));
+        Resources.subtractFrom(netAvailableRes,
+            tempAssigned.getCapacityAtTime(t));
+
+        // compute maximum number of gangs we could fit
+        curMaxGang =
+            (int) Math.floor(Resources.divide(plan.getResourceCalculator(),
+                totalCapacity, netAvailableRes, gang));
+
+        // pick the minimum between available resources in this instant, and how
+        // many gangs we have to place
+        curMaxGang = Math.min(gangsToPlace, curMaxGang);
+
+        // compare with previous max, and set it. also remember *where* we found
+        // the minimum (useful for next attempts)
+        if (curMaxGang <= maxGang) {
+          maxGang = curMaxGang;
+          minPoint = t;
+        }
+      }
+
+      // if we were able to place any gang, record this, and decrement
+      // gangsToPlace
+      if (maxGang > 0) {
+        gangsToPlace -= maxGang;
+
+        ReservationInterval reservationInt =
+            new ReservationInterval(curDeadline - dur, curDeadline);
+        ReservationRequest reservationRes =
+            ReservationRequest.newInstance(rr.getCapability(),
+                rr.getConcurrency() * maxGang, rr.getConcurrency(),
+                rr.getDuration());
+        // remember occupied space (plan is read-only till we find a plausible
+        // allocation for the entire request). This is needed since we might be
+        // placing other ReservationRequest within the same
+        // ReservationDefinition,
+        // and we must avoid double-counting the available resources
+        tempAssigned.addInterval(reservationInt, reservationRes);
+        allocationRequests.put(reservationInt, reservationRes);
+
+      }
+
+      // reset our new starting point (curDeadline) to the most constraining
+      // point so far, we will look "left" of that to find more places where
+      // to schedule gangs (for sure nothing on the "right" of this point can
+      // fit a full gang.
+      curDeadline = minPoint;
+    }
+
+    // if no gangs are left to place we succeed and return the allocation
+    if (gangsToPlace == 0) {
+      return allocationRequests;
+    } else {
+      // 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 :
+        allocationRequests.entrySet()) {
+        tempAssigned.removeInterval(tempAllocation.getKey(),
+            tempAllocation.getValue());
+      }
+      // and return null to signal failure in this allocation
+      return null;
+    }
+  }
+
+  // finds the leftmost point of this set of ReservationInterval
+  private long findEarliestTime(Set<ReservationInterval> resInt) {
+    long ret = Long.MAX_VALUE;
+    for (ReservationInterval s : resInt) {
+      if (s.getStartTime() < ret) {
+        ret = s.getStartTime();
+      }
+    }
+    return ret;
+  }
+
+  // finds the rightmost point of this set of ReservationIntervals
+  private long findLatestTime(Set<ReservationInterval> resInt) {
+    long ret = Long.MIN_VALUE;
+    for (ReservationInterval s : resInt) {
+      if (s.getEndTime() > ret) {
+        ret = s.getEndTime();
+      }
+    }
+    return ret;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66ffcf8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAgent.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/ReservationAgent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationAgent.java
new file mode 100644
index 0000000..fe1941d
--- /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/ReservationAgent.java
@@ -0,0 +1,55 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+
+/**
+ * An entity that seeks to acquire resources to satisfy an user's contract
+ */
+public interface ReservationAgent {
+
+  /**
+   * Create a reservation for the user that abides by the specified contract
+   * 
+   * @param reservationId the identifier of the reservation to be created.
+   * @param user the user who wants to create the reservation
+   * @param plan the Plan to which the reservation must be fitted
+   * @param contract encapsulates the resources the user requires for his
+   *          session
+   * 
+   * @return whether the create operation was successful or not
+   * @throws PlanningException if the session cannot be fitted into the plan
+   */
+  public boolean createReservation(ReservationId reservationId, String user,
+      Plan plan, ReservationDefinition contract) throws PlanningException;
+
+  /**
+   * Update a reservation for the user that abides by the specified contract
+   * 
+   * @param reservationId the identifier of the reservation to be updated
+   * @param user the user who wants to create the session
+   * @param plan the Plan to which the reservation must be fitted
+   * @param contract encapsulates the resources the user requires for his
+   *          reservation
+   * 
+   * @return whether the update operation was successful or not
+   * @throws PlanningException if the reservation cannot be fitted into the plan
+   */
+  public boolean updateReservation(ReservationId reservationId, String user,
+      Plan plan, ReservationDefinition contract) throws PlanningException;
+
+  /**
+   * Delete an user reservation
+   * 
+   * @param reservationId the identifier of the reservation to be deleted
+   * @param user the user who wants to create the reservation
+   * @param plan the Plan to which the session must be fitted
+   * 
+   * @return whether the delete operation was successful or not
+   * @throws PlanningException if the reservation cannot be fitted into the plan
+   */
+  public boolean deleteReservation(ReservationId reservationId, String user,
+      Plan plan) throws PlanningException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66ffcf8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ContractValidationException.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/exceptions/ContractValidationException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/ContractValidationException.java
new file mode 100644
index 0000000..7ee5a76
--- /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/exceptions/ContractValidationException.java
@@ -0,0 +1,12 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions;
+
+public class ContractValidationException extends PlanningException {
+
+  private static final long serialVersionUID = 1L;
+
+  public ContractValidationException(String message) {
+    super(message);
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66ffcf8/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
new file mode 100644
index 0000000..0b0201d
--- /dev/null
+++ 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
@@ -0,0 +1,588 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+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.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationDefinitionPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationRequestsPBImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Before;
+import org.junit.Test;
+import org.mortbay.log.Log;
+
+public class TestGreedyReservationAgent {
+
+  ReservationAgent agent;
+  InMemoryPlan plan;
+  Resource minAlloc = Resource.newInstance(1024, 1);
+  ResourceCalculator res = new DefaultResourceCalculator();
+  Resource maxAlloc = Resource.newInstance(1024 * 8, 8);
+  Random rand = new Random();
+  long step;
+
+  @Before
+  public void setup() throws Exception {
+
+    long seed = rand.nextLong();
+    rand.setSeed(seed);
+    Log.info("Running with seed: " + seed);
+
+    // setting completely loose quotas
+    long timeWindow = 1000000L;
+    Resource clusterCapacity = Resource.newInstance(100 * 1024, 100);
+    step = 1000L;
+    ReservationSystemTestUtil testUtil = new ReservationSystemTestUtil();
+    CapacityScheduler scheduler = testUtil.mockCapacityScheduler(125);
+    String reservationQ = testUtil.getFullReservationQueueName();
+    CapacitySchedulerConfiguration capConf = scheduler.getConfiguration();
+    capConf.setReservationWindow(reservationQ, timeWindow);
+    capConf.setMaximumCapacity(reservationQ, 100);
+    capConf.setAverageCapacity(reservationQ, 100);
+    CapacityOverTimePolicy policy = new CapacityOverTimePolicy();
+    policy.init(reservationQ, capConf, new HashSet<String>());
+    agent = new GreedyReservationAgent();
+
+    QueueMetrics queueMetrics = QueueMetrics.forQueue("dedicated",
+        mock(ParentQueue.class), false, capConf);
+
+    plan = new InMemoryPlan(queueMetrics, policy, agent, clusterCapacity, step,
+        res, minAlloc, maxAlloc, "dedicated", null, true);
+  }
+
+  @SuppressWarnings("javadoc")
+  @Test
+  public void testSimple() throws PlanningException {
+
+    prepareBasicPlan();
+
+    // create a request with a single atomic ask
+    ReservationDefinition rr = new ReservationDefinitionPBImpl();
+    rr.setArrival(5 * step);
+    rr.setDeadline(20 * step);
+    ReservationRequest r = ReservationRequest.newInstance(
+        Resource.newInstance(2048, 2), 10, 5, 10 * step);
+    ReservationRequests reqs = new ReservationRequestsPBImpl();
+    reqs.setReservationResources(Collections.singletonList(r));
+    rr.setReservationRequests(reqs);
+
+    ReservationId reservationID = ReservationSystemTestUtil
+        .getNewReservationId();
+    agent.createReservation(reservationID, "u1", plan, rr);
+
+    assertTrue("Agent-based allocation failed", reservationID != null);
+    assertTrue("Agent-based allocation failed", plan.getAllReservations()
+        .size() == 3);
+
+    ReservationAllocation cs = plan.getReservationById(reservationID);
+
+    System.out.println("--------AFTER SIMPLE ALLOCATION (queue: "
+        + reservationID + ")----------");
+    System.out.println(plan.toString());
+    System.out.println(plan.toCumulativeString());
+
+    for (long i = 10 * step; i < 20 * step; i++) {
+      assertTrue(
+          "Agent-based allocation unexpected",
+          Resources.equals(cs.getResourcesAtTime(i),
+              Resource.newInstance(2048 * 10, 2 * 10)));
+    }
+
+  }
+
+  @Test
+  public void testOrder() throws PlanningException {
+    prepareBasicPlan();
+
+    // create a completely utilized segment around time 30
+    int[] f = { 100, 100 };
+
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(
+            ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+            "dedicated", 30 * step, 30 * step + f.length * step,
+            ReservationSystemTestUtil.generateAllocation(30 * step, step, f),
+            res, minAlloc)));
+
+    // create a chain of 4 RR, mixing gang and non-gang
+    ReservationDefinition rr = new ReservationDefinitionPBImpl();
+    rr.setArrival(0 * step);
+    rr.setDeadline(70 * step);
+    ReservationRequests reqs = new ReservationRequestsPBImpl();
+    reqs.setInterpreter(ReservationRequestInterpreter.R_ORDER);
+    ReservationRequest r = ReservationRequest.newInstance(
+        Resource.newInstance(2048, 2), 10, 1, 10 * step);
+    ReservationRequest r2 = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), 10, 10, 20 * step);
+    List<ReservationRequest> list = new ArrayList<ReservationRequest>();
+    list.add(r);
+    list.add(r2);
+    list.add(r);
+    list.add(r2);
+    reqs.setReservationResources(list);
+    rr.setReservationRequests(reqs);
+
+    // submit to agent
+    ReservationId reservationID = ReservationSystemTestUtil
+        .getNewReservationId();
+    agent.createReservation(reservationID, "u1", plan, rr);
+
+    // validate
+    assertTrue("Agent-based allocation failed", reservationID != null);
+    assertTrue("Agent-based allocation failed", plan.getAllReservations()
+        .size() == 4);
+
+    ReservationAllocation cs = plan.getReservationById(reservationID);
+
+    assertTrue(cs.toString(), check(cs, 0 * step, 10 * step, 20, 1024, 1));
+    assertTrue(cs.toString(), check(cs, 10 * step, 30 * step, 10, 1024, 1));
+    assertTrue(cs.toString(), check(cs, 40 * step, 50 * step, 20, 1024, 1));
+    assertTrue(cs.toString(), check(cs, 50 * step, 70 * step, 10, 1024, 1));
+
+    System.out.println("--------AFTER ORDER ALLOCATION (queue: "
+        + reservationID + ")----------");
+    System.out.println(plan.toString());
+    System.out.println(plan.toCumulativeString());
+
+  }
+
+  @Test
+  public void testOrderNoGapImpossible() throws PlanningException {
+    prepareBasicPlan();
+    // create a completely utilized segment at time 30
+    int[] f = { 100, 100 };
+
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(
+            ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+            "dedicated", 30 * step, 30 * step + f.length * step,
+            ReservationSystemTestUtil.generateAllocation(30 * step, step, f),
+            res, minAlloc)));
+
+    // create a chain of 4 RR, mixing gang and non-gang
+    ReservationDefinition rr = new ReservationDefinitionPBImpl();
+    rr.setArrival(0L);
+
+    rr.setDeadline(70L);
+    ReservationRequests reqs = new ReservationRequestsPBImpl();
+    reqs.setInterpreter(ReservationRequestInterpreter.R_ORDER_NO_GAP);
+    ReservationRequest r = ReservationRequest.newInstance(
+        Resource.newInstance(2048, 2), 10, 1, 10);
+    ReservationRequest r2 = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), 10, 10, 20);
+    List<ReservationRequest> list = new ArrayList<ReservationRequest>();
+    list.add(r);
+    list.add(r2);
+    list.add(r);
+    list.add(r2);
+    reqs.setReservationResources(list);
+    rr.setReservationRequests(reqs);
+
+    ReservationId reservationID = ReservationSystemTestUtil
+        .getNewReservationId();
+    boolean result = false;
+    try {
+      // submit to agent
+      result = agent.createReservation(reservationID, "u1", plan, rr);
+      fail();
+    } catch (PlanningException p) {
+      // expected
+    }
+
+    // validate
+    assertFalse("Agent-based allocation should have failed", result);
+    assertTrue("Agent-based allocation should have failed", plan
+        .getAllReservations().size() == 3);
+
+    System.out
+        .println("--------AFTER ORDER_NO_GAP IMPOSSIBLE ALLOCATION (queue: "
+            + reservationID + ")----------");
+    System.out.println(plan.toString());
+    System.out.println(plan.toCumulativeString());
+
+  }
+
+  @Test
+  public void testOrderNoGap() throws PlanningException {
+    prepareBasicPlan();
+    // create a chain of 4 RR, mixing gang and non-gang
+    ReservationDefinition rr = new ReservationDefinitionPBImpl();
+    rr.setArrival(0 * step);
+    rr.setDeadline(60 * step);
+    ReservationRequests reqs = new ReservationRequestsPBImpl();
+    reqs.setInterpreter(ReservationRequestInterpreter.R_ORDER_NO_GAP);
+    ReservationRequest r = ReservationRequest.newInstance(
+        Resource.newInstance(2048, 2), 10, 1, 10 * step);
+    ReservationRequest r2 = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), 10, 10, 20 * step);
+    List<ReservationRequest> list = new ArrayList<ReservationRequest>();
+    list.add(r);
+    list.add(r2);
+    list.add(r);
+    list.add(r2);
+    reqs.setReservationResources(list);
+    rr.setReservationRequests(reqs);
+    rr.setReservationRequests(reqs);
+
+    // submit to agent
+    ReservationId reservationID = ReservationSystemTestUtil
+        .getNewReservationId();
+    agent.createReservation(reservationID, "u1", plan, rr);
+
+    System.out.println("--------AFTER ORDER ALLOCATION (queue: "
+        + reservationID + ")----------");
+    System.out.println(plan.toString());
+    System.out.println(plan.toCumulativeString());
+
+    // validate
+    assertTrue("Agent-based allocation failed", reservationID != null);
+    assertTrue("Agent-based allocation failed", plan.getAllReservations()
+        .size() == 3);
+
+    ReservationAllocation cs = plan.getReservationById(reservationID);
+
+    assertTrue(cs.toString(), check(cs, 0 * step, 10 * step, 20, 1024, 1));
+    assertTrue(cs.toString(), check(cs, 10 * step, 30 * step, 10, 1024, 1));
+    assertTrue(cs.toString(), check(cs, 30 * step, 40 * step, 20, 1024, 1));
+    assertTrue(cs.toString(), check(cs, 40 * step, 60 * step, 10, 1024, 1));
+
+  }
+
+  @Test
+  public void testSingleSliding() throws PlanningException {
+    prepareBasicPlan();
+
+    // create a single request for which we need subsequent (tight) packing.
+    ReservationDefinition rr = new ReservationDefinitionPBImpl();
+    rr.setArrival(100 * step);
+    rr.setDeadline(120 * step);
+    ReservationRequests reqs = new ReservationRequestsPBImpl();
+    reqs.setInterpreter(ReservationRequestInterpreter.R_ALL);
+    ReservationRequest r = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), 200, 10, 10 * step);
+
+    List<ReservationRequest> list = new ArrayList<ReservationRequest>();
+    list.add(r);
+    reqs.setReservationResources(list);
+    rr.setReservationRequests(reqs);
+
+    // submit to agent
+    ReservationId reservationID = ReservationSystemTestUtil
+        .getNewReservationId();
+    agent.createReservation(reservationID, "u1", plan, rr);
+
+    // validate results, we expect the second one to be accepted
+    assertTrue("Agent-based allocation failed", reservationID != null);
+    assertTrue("Agent-based allocation failed", plan.getAllReservations()
+        .size() == 3);
+
+    ReservationAllocation cs = plan.getReservationById(reservationID);
+
+    assertTrue(cs.toString(), check(cs, 100 * step, 120 * step, 100, 1024, 1));
+
+    System.out.println("--------AFTER packed ALLOCATION (queue: "
+        + reservationID + ")----------");
+    System.out.println(plan.toString());
+    System.out.println(plan.toCumulativeString());
+
+  }
+
+  @Test
+  public void testAny() throws PlanningException {
+    prepareBasicPlan();
+    // create an ANY request, with an impossible step (last in list, first
+    // considered),
+    // and two satisfiable ones. We expect the second one to be returned.
+
+    ReservationDefinition rr = new ReservationDefinitionPBImpl();
+    rr.setArrival(100 * step);
+    rr.setDeadline(120 * step);
+    ReservationRequests reqs = new ReservationRequestsPBImpl();
+    reqs.setInterpreter(ReservationRequestInterpreter.R_ANY);
+    ReservationRequest r = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), 5, 5, 10 * step);
+    ReservationRequest r2 = ReservationRequest.newInstance(
+        Resource.newInstance(2048, 2), 10, 5, 10 * step);
+    ReservationRequest r3 = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), 110, 110, 10 * step);
+
+    List<ReservationRequest> list = new ArrayList<ReservationRequest>();
+    list.add(r);
+    list.add(r2);
+    list.add(r3);
+    reqs.setReservationResources(list);
+    rr.setReservationRequests(reqs);
+
+    // submit to agent
+    ReservationId reservationID = ReservationSystemTestUtil
+        .getNewReservationId();
+    boolean res = agent.createReservation(reservationID, "u1", plan, rr);
+
+    // validate results, we expect the second one to be accepted
+    assertTrue("Agent-based allocation failed", res);
+    assertTrue("Agent-based allocation failed", plan.getAllReservations()
+        .size() == 3);
+
+    ReservationAllocation cs = plan.getReservationById(reservationID);
+
+    assertTrue(cs.toString(), check(cs, 110 * step, 120 * step, 20, 1024, 1));
+
+    System.out.println("--------AFTER ANY ALLOCATION (queue: " + reservationID
+        + ")----------");
+    System.out.println(plan.toString());
+    System.out.println(plan.toCumulativeString());
+
+  }
+
+  @Test
+  public void testAnyImpossible() throws PlanningException {
+    prepareBasicPlan();
+    // create an ANY request, with all impossible alternatives
+    ReservationDefinition rr = new ReservationDefinitionPBImpl();
+    rr.setArrival(100L);
+    rr.setDeadline(120L);
+    ReservationRequests reqs = new ReservationRequestsPBImpl();
+    reqs.setInterpreter(ReservationRequestInterpreter.R_ANY);
+
+    // longer than arrival-deadline
+    ReservationRequest r1 = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), 35, 5, 30);
+    // above max cluster size
+    ReservationRequest r2 = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), 110, 110, 10);
+
+    List<ReservationRequest> list = new ArrayList<ReservationRequest>();
+    list.add(r1);
+    list.add(r2);
+    reqs.setReservationResources(list);
+    rr.setReservationRequests(reqs);
+
+    ReservationId reservationID = ReservationSystemTestUtil
+        .getNewReservationId();
+    boolean result = false;
+    try {
+      // submit to agent
+      result = agent.createReservation(reservationID, "u1", plan, rr);
+      fail();
+    } catch (PlanningException p) {
+      // expected
+    }
+    // validate results, we expect the second one to be accepted
+    assertFalse("Agent-based allocation should have failed", result);
+    assertTrue("Agent-based allocation should have failed", plan
+        .getAllReservations().size() == 2);
+
+    System.out.println("--------AFTER ANY IMPOSSIBLE ALLOCATION (queue: "
+        + reservationID + ")----------");
+    System.out.println(plan.toString());
+    System.out.println(plan.toCumulativeString());
+
+  }
+
+  @Test
+  public void testAll() throws PlanningException {
+    prepareBasicPlan();
+    // create an ALL request
+    ReservationDefinition rr = new ReservationDefinitionPBImpl();
+    rr.setArrival(100 * step);
+    rr.setDeadline(120 * step);
+    ReservationRequests reqs = new ReservationRequestsPBImpl();
+    reqs.setInterpreter(ReservationRequestInterpreter.R_ALL);
+    ReservationRequest r = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), 5, 5, 10 * step);
+    ReservationRequest r2 = ReservationRequest.newInstance(
+        Resource.newInstance(2048, 2), 10, 10, 20 * step);
+
+    List<ReservationRequest> list = new ArrayList<ReservationRequest>();
+    list.add(r);
+    list.add(r2);
+    reqs.setReservationResources(list);
+    rr.setReservationRequests(reqs);
+
+    // submit to agent
+    ReservationId reservationID = ReservationSystemTestUtil
+        .getNewReservationId();
+    agent.createReservation(reservationID, "u1", plan, rr);
+
+    // validate results, we expect the second one to be accepted
+    assertTrue("Agent-based allocation failed", reservationID != null);
+    assertTrue("Agent-based allocation failed", plan.getAllReservations()
+        .size() == 3);
+
+    ReservationAllocation cs = plan.getReservationById(reservationID);
+
+    assertTrue(cs.toString(), check(cs, 100 * step, 110 * step, 20, 1024, 1));
+    assertTrue(cs.toString(), check(cs, 110 * step, 120 * step, 25, 1024, 1));
+
+    System.out.println("--------AFTER ALL ALLOCATION (queue: " + reservationID
+        + ")----------");
+    System.out.println(plan.toString());
+    System.out.println(plan.toCumulativeString());
+
+  }
+
+  @Test
+  public void testAllImpossible() throws PlanningException {
+    prepareBasicPlan();
+    // create an ALL request, with an impossible combination, it should be
+    // rejected, and allocation remain unchanged
+    ReservationDefinition rr = new ReservationDefinitionPBImpl();
+    rr.setArrival(100L);
+    rr.setDeadline(120L);
+    ReservationRequests reqs = new ReservationRequestsPBImpl();
+    reqs.setInterpreter(ReservationRequestInterpreter.R_ALL);
+    ReservationRequest r = ReservationRequest.newInstance(
+        Resource.newInstance(1024, 1), 55, 5, 10);
+    ReservationRequest r2 = ReservationRequest.newInstance(
+        Resource.newInstance(2048, 2), 55, 5, 20);
+
+    List<ReservationRequest> list = new ArrayList<ReservationRequest>();
+    list.add(r);
+    list.add(r2);
+    reqs.setReservationResources(list);
+    rr.setReservationRequests(reqs);
+
+    ReservationId reservationID = ReservationSystemTestUtil
+        .getNewReservationId();
+    boolean result = false;
+    try {
+      // submit to agent
+      result = agent.createReservation(reservationID, "u1", plan, rr);
+      fail();
+    } catch (PlanningException p) {
+      // expected
+    }
+
+    // validate results, we expect the second one to be accepted
+    assertFalse("Agent-based allocation failed", result);
+    assertTrue("Agent-based allocation failed", plan.getAllReservations()
+        .size() == 2);
+
+    System.out.println("--------AFTER ALL IMPOSSIBLE ALLOCATION (queue: "
+        + reservationID + ")----------");
+    System.out.println(plan.toString());
+    System.out.println(plan.toCumulativeString());
+
+  }
+
+  private void prepareBasicPlan() throws PlanningException {
+
+    // insert in the reservation a couple of controlled reservations, to create
+    // conditions for assignment that are non-empty
+
+    int[] f = { 10, 10, 20, 20, 20, 10, 10 };
+
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(
+            ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+            "dedicated", 0L, 0L + f.length * step, ReservationSystemTestUtil
+                .generateAllocation(0, step, f), res, minAlloc)));
+
+    int[] f2 = { 5, 5, 5, 5, 5, 5, 5 };
+    Map<ReservationInterval, ReservationRequest> alloc = 
+        ReservationSystemTestUtil.generateAllocation(5000, step, f2);
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(
+            ReservationSystemTestUtil.getNewReservationId(), null, "u1",
+            "dedicated", 5000, 5000 + f2.length * step, alloc, res, minAlloc)));
+
+    System.out.println("--------BEFORE AGENT----------");
+    System.out.println(plan.toString());
+    System.out.println(plan.toCumulativeString());
+  }
+
+  private boolean check(ReservationAllocation cs, long start, long end,
+      int containers, int mem, int cores) {
+
+    boolean res = true;
+    for (long i = start; i < end; i++) {
+      res = res
+          && Resources.equals(cs.getResourcesAtTime(i),
+              Resource.newInstance(mem * containers, cores * containers));
+    }
+    return res;
+  }
+
+  public void testStress(int numJobs) throws PlanningException, IOException {
+
+    long timeWindow = 1000000L;
+    Resource clusterCapacity = Resource.newInstance(500 * 100 * 1024, 500 * 32);
+    step = 1000L;
+    ReservationSystemTestUtil testUtil = new ReservationSystemTestUtil();
+    CapacityScheduler scheduler = testUtil.mockCapacityScheduler(500 * 100);
+    String reservationQ = testUtil.getFullReservationQueueName();
+    CapacitySchedulerConfiguration capConf = scheduler.getConfiguration();
+    capConf.setReservationWindow(reservationQ, timeWindow);
+    capConf.setMaximumCapacity(reservationQ, 100);
+    capConf.setAverageCapacity(reservationQ, 100);
+    CapacityOverTimePolicy policy = new CapacityOverTimePolicy();
+    policy.init(reservationQ, capConf, new HashSet<String>());
+
+    plan = new InMemoryPlan(scheduler.getRootQueueMetrics(), policy, agent,
+      clusterCapacity, step, res, minAlloc, maxAlloc, "dedicated", null, true);
+
+    int acc = 0;
+    List<ReservationDefinition> list = new ArrayList<ReservationDefinition>();
+    for (long i = 0; i < numJobs; i++) {
+      list.add(ReservationSystemTestUtil.generateRandomRR(rand, i));
+    }
+
+    long start = System.currentTimeMillis();
+    for (int i = 0; i < numJobs; i++) {
+
+      try {
+        if (agent.createReservation(
+            ReservationSystemTestUtil.getNewReservationId(), "u" + i % 100,
+            plan, list.get(i))) {
+          acc++;
+        }
+      } catch (PlanningException p) {
+        // ignore exceptions
+      }
+    }
+
+    long end = System.currentTimeMillis();
+    System.out.println("Submitted " + numJobs + " jobs " + " accepted " + acc
+        + " in " + (end - start) + "ms");
+  }
+
+  public static void main(String[] arg) {
+
+    // run a stress test with by default 1000 random jobs
+    int numJobs = 1000;
+    if (arg.length > 0) {
+      numJobs = Integer.parseInt(arg[0]);
+    }
+
+    try {
+      TestGreedyReservationAgent test = new TestGreedyReservationAgent();
+      test.setup();
+      test.testStress(numJobs);
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+}


[02/16] git commit: YARN-2475. Logic for responding to capacity drops for the ReservationSystem. Contributed by Carlo Curino and Subru Krishnan. (cherry picked from commit f83a07f266f2c5e6eead554d8a331ed7e75e10d5)

Posted by cd...@apache.org.
YARN-2475. Logic for responding to capacity drops for the ReservationSystem. Contributed by Carlo Curino and Subru Krishnan.
(cherry picked from commit f83a07f266f2c5e6eead554d8a331ed7e75e10d5)


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

Branch: refs/heads/trunk
Commit: 1c6950354f3c35a7824770dc251d5aec3be4876a
Parents: eb3e40b
Author: carlo curino <Carlo Curino>
Authored: Fri Sep 12 16:52:54 2014 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Oct 3 15:41:21 2014 -0700

----------------------------------------------------------------------
 YARN-1051-CHANGES.txt                           |   5 +
 .../reservation/SimpleCapacityReplanner.java    |  95 ++++++++++++
 .../TestSimpleCapacityReplanner.java            | 149 +++++++++++++++++++
 3 files changed, 249 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c695035/YARN-1051-CHANGES.txt
----------------------------------------------------------------------
diff --git a/YARN-1051-CHANGES.txt b/YARN-1051-CHANGES.txt
new file mode 100644
index 0000000..9fd4b3b
--- /dev/null
+++ b/YARN-1051-CHANGES.txt
@@ -0,0 +1,5 @@
+YARN-1707. Introduce APIs to add/remove/resize queues in the
+CapacityScheduler. (Carlo Curino and Subru Krishnan via curino)
+
+YARN-2475. Logic for responding to capacity drops for the 
+ReservationSystem. (Carlo Curino and Subru Krishnan via curino)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c695035/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SimpleCapacityReplanner.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/SimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SimpleCapacityReplanner.java
new file mode 100644
index 0000000..8384538
--- /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/SimpleCapacityReplanner.java
@@ -0,0 +1,95 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+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.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.UTCClock;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This (re)planner scan a period of time from now to a maximum time window (or
+ * the end of the last session, whichever comes first) checking the overall
+ * capacity is not violated.
+ * 
+ * It greedily removes sessions in reversed order of acceptance (latest accepted
+ * is the first removed).
+ */
+public class SimpleCapacityReplanner implements Planner {
+
+  private static final Log LOG = LogFactory
+      .getLog(SimpleCapacityReplanner.class);
+
+  private static final Resource ZERO_RESOURCE = Resource.newInstance(0, 0);
+
+  private final Clock clock;
+
+  // this allows to control to time-span of this replanning
+  // far into the future time instants might be worth replanning for
+  // later on
+  private long lengthOfCheckZone;
+
+  public SimpleCapacityReplanner() {
+    this(new UTCClock());
+  }
+
+  @VisibleForTesting
+  SimpleCapacityReplanner(Clock clock) {
+    this.clock = clock;
+  }
+
+  @Override
+  public void init(String planQueueName, CapacitySchedulerConfiguration conf) {
+    this.lengthOfCheckZone = conf.getEnforcementWindow(planQueueName);
+  }
+
+  @Override
+  public void plan(Plan plan, List<ReservationDefinition> contracts)
+      throws PlanningException {
+
+    if (contracts != null) {
+      throw new RuntimeException(
+          "SimpleCapacityReplanner cannot handle new reservation contracts");
+    }
+
+    ResourceCalculator resCalc = plan.getResourceCalculator();
+    Resource totCap = plan.getTotalCapacity();
+    long now = clock.getTime();
+
+    // loop on all moment in time from now to the end of the check Zone
+    // or the end of the planned sessions whichever comes first
+    for (long t = now; (t < plan.getLastEndTime() && t < (now + lengthOfCheckZone)); t +=
+        plan.getStep()) {
+      Resource excessCap =
+          Resources.subtract(plan.getTotalCommittedResources(t), totCap);
+      // if we are violating
+      if (Resources.greaterThan(resCalc, totCap, excessCap, ZERO_RESOURCE)) {
+        // sorted on reverse order of acceptance, so newest reservations first
+        Set<ReservationAllocation> curReservations =
+            new TreeSet<ReservationAllocation>(plan.getReservationsAtTime(t));
+        for (Iterator<ReservationAllocation> resIter =
+            curReservations.iterator(); resIter.hasNext()
+            && Resources.greaterThan(resCalc, totCap, excessCap, ZERO_RESOURCE);) {
+          ReservationAllocation reservation = resIter.next();
+          plan.deleteReservation(reservation.getReservationId());
+          excessCap =
+              Resources.subtract(excessCap, reservation.getResourcesAtTime(t));
+          LOG.info("Removing reservation " + reservation.getReservationId()
+              + " to repair physical-resource constraints in the plan: "
+              + plan.getQueueName());
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c695035/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
new file mode 100644
index 0000000..f2313e6
--- /dev/null
+++ 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
@@ -0,0 +1,149 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.TreeMap;
+
+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;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.junit.Test;
+
+public class TestSimpleCapacityReplanner {
+
+  @Test
+  public void testReplanningPlanCapacityLoss() throws PlanningException {
+
+    Resource clusterCapacity = Resource.newInstance(100 * 1024, 10);
+    Resource minAlloc = Resource.newInstance(1024, 1);
+    Resource maxAlloc = Resource.newInstance(1024 * 8, 8);
+
+    ResourceCalculator res = new DefaultResourceCalculator();
+    long step = 1L;
+    Clock clock = mock(Clock.class);
+    ReservationAgent agent = mock(ReservationAgent.class);
+
+    SharingPolicy policy = new NoOverCommitPolicy();
+    policy.init("root.dedicated", null, new HashSet<String>());
+
+    QueueMetrics queueMetrics = mock(QueueMetrics.class);
+
+    when(clock.getTime()).thenReturn(0L);
+    SimpleCapacityReplanner enf = new SimpleCapacityReplanner(clock);
+
+    CapacitySchedulerConfiguration conf =
+        mock(CapacitySchedulerConfiguration.class);
+    when(conf.getEnforcementWindow(any(String.class))).thenReturn(6L);
+
+    conf.setLong(CapacitySchedulerConfiguration.PREFIX + "blah"
+        + CapacitySchedulerConfiguration.DOT
+        + CapacitySchedulerConfiguration.RESERVATION_ENFORCEMENT_WINDOW, 6);
+    enf.init("blah", conf);
+
+    // Initialize the plan with more resources
+    InMemoryPlan plan =
+        new InMemoryPlan(queueMetrics, policy, agent, clusterCapacity, step,
+            res, minAlloc, maxAlloc, "dedicated", enf, true, clock);
+
+    // add reservation filling the plan (separating them 1ms, so we are sure
+    // s2 follows s1 on acceptance
+    long ts = System.currentTimeMillis();
+    ReservationId r1 = ReservationId.newInstance(ts, 1);
+    int[] f5 = { 20, 20, 20, 20, 20 };
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(r1, null, "u3",
+            "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
+            minAlloc)));
+    when(clock.getTime()).thenReturn(1L);
+    ReservationId r2 = ReservationId.newInstance(ts, 2);
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(r2, null, "u4",
+            "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
+            minAlloc)));
+    when(clock.getTime()).thenReturn(2L);
+    ReservationId r3 = ReservationId.newInstance(ts, 3);
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(r3, null, "u5",
+            "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
+            minAlloc)));
+    when(clock.getTime()).thenReturn(3L);
+    ReservationId r4 = ReservationId.newInstance(ts, 4);
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(r4, null, "u6",
+            "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
+            minAlloc)));
+    when(clock.getTime()).thenReturn(4L);
+    ReservationId r5 = ReservationId.newInstance(ts, 5);
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(r5, null, "u7",
+            "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
+            minAlloc)));
+
+    int[] f6 = { 50, 50, 50, 50, 50 };
+    ReservationId r6 = ReservationId.newInstance(ts, 6);
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(r6, null, "u3",
+            "dedicated", 10, 10 + f6.length, generateAllocation(10, f6), res,
+            minAlloc)));
+    when(clock.getTime()).thenReturn(6L);
+    ReservationId r7 = ReservationId.newInstance(ts, 7);
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(r7, null, "u4",
+            "dedicated", 10, 10 + f6.length, generateAllocation(10, f6), res,
+            minAlloc)));
+
+    // remove some of the resources (requires replanning)
+    plan.setTotalCapacity(Resource.newInstance(70 * 1024, 70));
+
+    when(clock.getTime()).thenReturn(0L);
+
+    // run the replanner
+    enf.plan(plan, null);
+
+    // check which reservation are still present
+    assertNotNull(plan.getReservationById(r1));
+    assertNotNull(plan.getReservationById(r2));
+    assertNotNull(plan.getReservationById(r3));
+    assertNotNull(plan.getReservationById(r6));
+    assertNotNull(plan.getReservationById(r7));
+
+    // and which ones are removed
+    assertNull(plan.getReservationById(r4));
+    assertNull(plan.getReservationById(r5));
+
+    // check resources at each moment in time no more exceed capacity
+    for (int i = 0; i < 20; i++) {
+      int tot = 0;
+      for (ReservationAllocation r : plan.getReservationsAtTime(i)) {
+        tot = r.getResourcesAtTime(i).getMemory();
+      }
+      assertTrue(tot <= 70 * 1024);
+    }
+  }
+
+  private Map<ReservationInterval, ReservationRequest> generateAllocation(
+      int startTime, int[] alloc) {
+    Map<ReservationInterval, ReservationRequest> req =
+        new TreeMap<ReservationInterval, ReservationRequest>();
+    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]));
+    }
+    return req;
+  }
+
+}


[09/16] git commit: YARN-1712. Plan follower that synchronizes the current state of reservation subsystem with the scheduler. Contributed by Subru Krishnan and Carlo Curino. (cherry picked from commit 169085319b8b76641f8b9f6840a3fef06d221e2b)

Posted by cd...@apache.org.
YARN-1712. Plan follower that synchronizes the current state of reservation subsystem with the scheduler. Contributed by Subru Krishnan and Carlo Curino.
(cherry picked from commit 169085319b8b76641f8b9f6840a3fef06d221e2b)


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

Branch: refs/heads/trunk
Commit: 3418c56bcf4bbddaf483bdaa1a15a8bbc4039bfe
Parents: b6df0dd
Author: subru <su...@outlook.com>
Authored: Tue Sep 16 16:45:45 2014 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Oct 3 15:42:10 2014 -0700

----------------------------------------------------------------------
 YARN-1051-CHANGES.txt                           |   7 +-
 .../CapacitySchedulerPlanFollower.java          | 367 +++++++++++++++++++
 .../reservation/PlanFollower.java               |  67 ++++
 .../TestCapacitySchedulerPlanFollower.java      | 319 ++++++++++++++++
 4 files changed, 758 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3418c56b/YARN-1051-CHANGES.txt
----------------------------------------------------------------------
diff --git a/YARN-1051-CHANGES.txt b/YARN-1051-CHANGES.txt
index e9ec691..56b3c12 100644
--- a/YARN-1051-CHANGES.txt
+++ b/YARN-1051-CHANGES.txt
@@ -5,10 +5,10 @@ YARN-2475. Logic for responding to capacity drops for the
 ReservationSystem. (Carlo Curino and Subru Krishnan via curino)
 
 YARN-1708. Public YARN APIs for creating/updating/deleting 
-reservations. (Carlo Curino and Subru Krishnan via subru)
+reservations. (Subru Krishnan and Carlo Curino  via subru)
 
 YARN-1709. In-memory data structures used to track resources over
-time to enable reservations. (Carlo Curino and Subru Krishnan via 
+time to enable reservations. (Subru Krishnan and Carlo Curino via 
 subru)
 
 YARN-1710. Logic to find allocations within a Plan that satisfy 
@@ -17,3 +17,6 @@ curino)
 
 YARN-1711. Policy to enforce instantaneous and over-time quotas 
 on user reservations. (Carlo Curino and Subru Krishnan via curino)
+
+YARN-1712. Plan follower that synchronizes the current state of reservation
+subsystem with the scheduler. (Subru Krishnan and Carlo Curino  via subru)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3418c56b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.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/CapacitySchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java
new file mode 100644
index 0000000..cfa172c
--- /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/CapacitySchedulerPlanFollower.java
@@ -0,0 +1,367 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PlanQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ReservationQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class implements a {@link PlanFollower}. This is invoked on a timer, and
+ * it is in charge to publish the state of the {@link Plan}s to the underlying
+ * {@link CapacityScheduler}. This implementation does so, by
+ * adding/removing/resizing leaf queues in the scheduler, thus affecting the
+ * dynamic behavior of the scheduler in a way that is consistent with the
+ * content of the plan. It also updates the plan's view on how much resources
+ * are available in the cluster.
+ * 
+ * This implementation of PlanFollower is relatively stateless, and it can
+ * synchronize schedulers and Plans that have arbitrary changes (performing set
+ * differences among existing queues). This makes it resilient to frequency of
+ * synchronization, and RM restart issues (no "catch up" is necessary).
+ */
+public class CapacitySchedulerPlanFollower implements PlanFollower {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(CapacitySchedulerPlanFollower.class);
+
+  private Collection<Plan> plans = new ArrayList<Plan>();
+
+  private Clock clock;
+  private CapacityScheduler scheduler;
+
+  @Override
+  public void init(Clock clock, ResourceScheduler sched, Collection<Plan> plans) {
+    LOG.info("Initializing Plan Follower Policy:"
+        + this.getClass().getCanonicalName());
+    if (!(sched instanceof CapacityScheduler)) {
+      throw new YarnRuntimeException(
+          "CapacitySchedulerPlanFollower can only work with CapacityScheduler");
+    }
+    this.clock = clock;
+    this.scheduler = (CapacityScheduler) sched;
+    this.plans.addAll(plans);
+  }
+
+  @Override
+  public synchronized void run() {
+    for (Plan plan : plans) {
+      synchronizePlan(plan);
+    }
+  }
+
+  @Override
+  public synchronized void synchronizePlan(Plan plan) {
+    String planQueueName = plan.getQueueName();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Running plan follower edit policy for plan: " + planQueueName);
+    }
+    // align with plan step
+    long step = plan.getStep();
+    long now = clock.getTime();
+    if (now % step != 0) {
+      now += step - (now % step);
+    }
+    CSQueue queue = scheduler.getQueue(planQueueName);
+    if (!(queue instanceof PlanQueue)) {
+      LOG.error("The Plan is not an PlanQueue!");
+      return;
+    }
+    PlanQueue planQueue = (PlanQueue) queue;
+    // first we publish to the plan the current availability of resources
+    Resource clusterResources = scheduler.getClusterResource();
+    float planAbsCap = planQueue.getAbsoluteCapacity();
+    Resource planResources = Resources.multiply(clusterResources, planAbsCap);
+    plan.setTotalCapacity(planResources);
+
+    Set<ReservationAllocation> currentReservations =
+        plan.getReservationsAtTime(now);
+    Set<String> curReservationNames = new HashSet<String>();
+    Resource reservedResources = Resource.newInstance(0, 0);
+    int numRes = 0;
+    if (currentReservations != null) {
+      numRes = currentReservations.size();
+      for (ReservationAllocation reservation : currentReservations) {
+        curReservationNames.add(reservation.getReservationId().toString());
+        Resources.addTo(reservedResources, reservation.getResourcesAtTime(now));
+      }
+    }
+    // create the default reservation queue if it doesnt exist
+    String defReservationQueue = planQueueName + PlanQueue.DEFAULT_QUEUE_SUFFIX;
+    if (scheduler.getQueue(defReservationQueue) == null) {
+      ReservationQueue defQueue =
+          new ReservationQueue(scheduler, defReservationQueue, planQueue);
+      try {
+        scheduler.addQueue(defQueue);
+      } catch (SchedulerDynamicEditException e) {
+        LOG.warn(
+            "Exception while trying to create default reservation queue for plan: {}",
+            planQueueName, e);
+      }
+    }
+    curReservationNames.add(defReservationQueue);
+    // if the resources dedicated to this plan has shrunk invoke replanner
+    if (Resources.greaterThan(scheduler.getResourceCalculator(),
+        clusterResources, reservedResources, planResources)) {
+      try {
+        plan.getReplanner().plan(plan, null);
+      } catch (PlanningException e) {
+        LOG.warn("Exception while trying to replan: {}", planQueueName, e);
+      }
+    }
+    // identify the reservations that have expired and new reservations that
+    // have to be activated
+    List<CSQueue> resQueues = planQueue.getChildQueues();
+    Set<String> expired = new HashSet<String>();
+    for (CSQueue resQueue : resQueues) {
+      String resQueueName = resQueue.getQueueName();
+      if (curReservationNames.contains(resQueueName)) {
+        // it is already existing reservation, so needed not create new
+        // reservation queue
+        curReservationNames.remove(resQueueName);
+      } else {
+        // the reservation has termination, mark for cleanup
+        expired.add(resQueueName);
+      }
+    }
+    // garbage collect expired reservations
+    cleanupExpiredQueues(plan.getMoveOnExpiry(), expired, defReservationQueue);
+
+    // Add new reservations and update existing ones
+    float totalAssignedCapacity = 0f;
+    if (currentReservations != null) {
+      // first release all excess capacity in default queue
+      try {
+        scheduler.setEntitlement(defReservationQueue, new QueueEntitlement(0f,
+            1.0f));
+      } catch (YarnException e) {
+        LOG.warn(
+            "Exception while trying to release default queue capacity for plan: {}",
+            planQueueName, e);
+      }
+      // sort allocations from the one giving up the most resources, to the
+      // one asking for the most
+      // avoid order-of-operation errors that temporarily violate 100%
+      // capacity bound
+      List<ReservationAllocation> sortedAllocations =
+          sortByDelta(
+              new ArrayList<ReservationAllocation>(currentReservations), now);
+      for (ReservationAllocation res : sortedAllocations) {
+        String currResId = res.getReservationId().toString();
+        if (curReservationNames.contains(currResId)) {
+          ReservationQueue resQueue =
+              new ReservationQueue(scheduler, currResId, planQueue);
+          try {
+            scheduler.addQueue(resQueue);
+          } catch (SchedulerDynamicEditException e) {
+            LOG.warn(
+                "Exception while trying to activate reservation: {} for plan: {}",
+                currResId, planQueueName, e);
+          }
+        }
+        Resource capToAssign = res.getResourcesAtTime(now);
+        float targetCapacity = 0f;
+        if (planResources.getMemory() > 0
+            && planResources.getVirtualCores() > 0) {
+          targetCapacity =
+              Resources.divide(scheduler.getResourceCalculator(),
+                  clusterResources, capToAssign, planResources);
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(
+              "Assigning capacity of {} to queue {} with target capacity {}",
+              capToAssign, currResId, targetCapacity);
+        }
+        // set maxCapacity to 100% unless the job requires gang, in which
+        // case we stick to capacity (as running early/before is likely a
+        // waste of resources)
+        float maxCapacity = 1.0f;
+        if (res.containsGangs()) {
+          maxCapacity = targetCapacity;
+        }
+        try {
+          scheduler.setEntitlement(currResId, new QueueEntitlement(
+              targetCapacity, maxCapacity));
+        } catch (YarnException e) {
+          LOG.warn("Exception while trying to size reservation for plan: {}",
+              currResId, planQueueName, e);
+        }
+        totalAssignedCapacity += targetCapacity;
+      }
+    }
+    // compute the default queue capacity
+    float defQCap = 1.0f - totalAssignedCapacity;
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("PlanFollowerEditPolicyTask: total Plan Capacity: {} "
+          + "currReservation: {} default-queue capacity: {}", planResources,
+          numRes, defQCap);
+    }
+    // set the default queue to eat-up all remaining capacity
+    try {
+      scheduler.setEntitlement(defReservationQueue, new QueueEntitlement(
+          defQCap, 1.0f));
+    } catch (YarnException e) {
+      LOG.warn(
+          "Exception while trying to reclaim default queue capacity for plan: {}",
+          planQueueName, e);
+    }
+    // garbage collect finished reservations from plan
+    try {
+      plan.archiveCompletedReservations(now);
+    } catch (PlanningException e) {
+      LOG.error("Exception in archiving completed reservations: ", e);
+    }
+    LOG.info("Finished iteration of plan follower edit policy for plan: "
+        + planQueueName);
+
+    // Extension: update plan with app states,
+    // useful to support smart replanning
+  }
+
+  /**
+   * Move all apps in the set of queues to the parent plan queue's default
+   * reservation queue in a synchronous fashion
+   */
+  private void moveAppsInQueueSync(String expiredReservation,
+      String defReservationQueue) {
+    List<ApplicationAttemptId> activeApps =
+        scheduler.getAppsInQueue(expiredReservation);
+    if (activeApps.isEmpty()) {
+      return;
+    }
+    for (ApplicationAttemptId app : activeApps) {
+      // fallback to parent's default queue
+      try {
+        scheduler.moveApplication(app.getApplicationId(), defReservationQueue);
+      } catch (YarnException e) {
+        LOG.warn(
+            "Encountered unexpected error during migration of application: {} from reservation: {}",
+            app, expiredReservation, e);
+      }
+    }
+  }
+
+  /**
+   * First sets entitlement of queues to zero to prevent new app submission.
+   * Then move all apps in the set of queues to the parent plan queue's default
+   * reservation queue if move is enabled. Finally cleanups the queue by killing
+   * any apps (if move is disabled or move failed) and removing the queue
+   */
+  private void cleanupExpiredQueues(boolean shouldMove, Set<String> toRemove,
+      String defReservationQueue) {
+    for (String expiredReservation : toRemove) {
+      try {
+        // reduce entitlement to 0
+        scheduler.setEntitlement(expiredReservation, new QueueEntitlement(0.0f,
+            0.0f));
+        if (shouldMove) {
+          moveAppsInQueueSync(expiredReservation, defReservationQueue);
+        }
+        if (scheduler.getAppsInQueue(expiredReservation).size() > 0) {
+          scheduler.killAllAppsInQueue(expiredReservation);
+          LOG.info("Killing applications in queue: {}", expiredReservation);
+        } else {
+          scheduler.removeQueue(expiredReservation);
+          LOG.info("Queue: " + expiredReservation + " removed");
+        }
+      } catch (YarnException e) {
+        LOG.warn("Exception while trying to expire reservation: {}",
+            expiredReservation, e);
+      }
+    }
+  }
+
+  @Override
+  public synchronized void setPlans(Collection<Plan> plans) {
+    this.plans.clear();
+    this.plans.addAll(plans);
+  }
+
+  /**
+   * Sort in the order from the least new amount of resources asked (likely
+   * negative) to the highest. This prevents "order-of-operation" errors related
+   * to exceeding 100% capacity temporarily.
+   */
+  private List<ReservationAllocation> sortByDelta(
+      List<ReservationAllocation> currentReservations, long now) {
+    Collections.sort(currentReservations, new ReservationAllocationComparator(
+        scheduler, now));
+    return currentReservations;
+  }
+
+  private class ReservationAllocationComparator implements
+      Comparator<ReservationAllocation> {
+    CapacityScheduler scheduler;
+    long now;
+
+    ReservationAllocationComparator(CapacityScheduler scheduler, long now) {
+      this.scheduler = scheduler;
+      this.now = now;
+    }
+
+    private Resource getUnallocatedReservedResources(
+        ReservationAllocation reservation) {
+      Resource resResource;
+      CSQueue resQueue =
+          scheduler.getQueue(reservation.getReservationId().toString());
+      if (resQueue != null) {
+        resResource =
+            Resources.subtract(
+                reservation.getResourcesAtTime(now),
+                Resources.multiply(scheduler.getClusterResource(),
+                    resQueue.getAbsoluteCapacity()));
+      } else {
+        resResource = reservation.getResourcesAtTime(now);
+      }
+      return resResource;
+    }
+
+    @Override
+    public int compare(ReservationAllocation lhs, ReservationAllocation rhs) {
+      // compute delta between current and previous reservation, and compare
+      // based on that
+      Resource lhsRes = getUnallocatedReservedResources(lhs);
+      Resource rhsRes = getUnallocatedReservedResources(rhs);
+      return lhsRes.compareTo(rhsRes);
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3418c56b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanFollower.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/PlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanFollower.java
new file mode 100644
index 0000000..9d00366
--- /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/PlanFollower.java
@@ -0,0 +1,67 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import java.util.Collection;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.util.Clock;
+
+/**
+ * A PlanFollower is a component that runs on a timer, and synchronizes the
+ * underlying {@link ResourceScheduler} with the {@link Plan}(s) and viceversa.
+ * 
+ * While different implementations might operate differently, the key idea is to
+ * map the current allocation of resources for each active reservation in the
+ * plan(s), to a corresponding notion in the underlying scheduler (e.g., tuning
+ * capacity of queues, set pool weights, or tweak application priorities). The
+ * goal is to affect the dynamic allocation of resources done by the scheduler
+ * so that the jobs obtain access to resources in a way that is consistent with
+ * the reservations in the plan. A key conceptual step here is to convert the
+ * absolute-valued promises made in the reservations to appropriate relative
+ * priorities/queue sizes etc.
+ * 
+ * Symmetrically the PlanFollower exposes changes in cluster conditions (as
+ * tracked by the scheduler) to the plan, e.g., the overall amount of physical
+ * resources available. The Plan in turn can react by replanning its allocations
+ * if appropriate.
+ * 
+ * The implementation can assume that is run frequently enough to be able to
+ * observe and react to normal operational changes in cluster conditions on the
+ * fly (e.g., if cluster resources drop, we can update the relative weights of a
+ * queue so that the absolute promises made to the job at reservation time are
+ * respected).
+ * 
+ * However, due to RM restarts and the related downtime, it is advisable for
+ * implementations to operate in a stateless way, and be able to synchronize the
+ * state of plans/scheduler regardless of how big is the time gap between
+ * executions.
+ */
+public interface PlanFollower extends Runnable {
+
+  /**
+   * Init function that configures the PlanFollower, by providing:
+   * 
+   * @param clock a reference to the system clock.
+   * @param sched a reference to the underlying scheduler
+   * @param plans references to the plans we should keep synchronized at every
+   *          time tick.
+   */
+  public void init(Clock clock, ResourceScheduler sched, Collection<Plan> plans);
+
+  /**
+   * The function performing the actual synchronization operation for a given
+   * Plan. This is normally invoked by the run method, but it can be invoked
+   * synchronously to avoid race conditions when a user's reservation request
+   * start time is imminent.
+   * 
+   * @param plan the Plan to synchronize
+   */
+  public void synchronizePlan(Plan plan);
+
+  /**
+   * Setter for the list of plans.
+   * 
+   * @param plans the collection of Plans we operate on at every time tick.
+   */
+  public void setPlans(Collection<Plan> plans);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3418c56b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.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/TestCapacitySchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java
new file mode 100644
index 0000000..4eedd42
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java
@@ -0,0 +1,319 @@
+/**
+ * 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 static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+import java.util.Collections;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PlanQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+
+public class TestCapacitySchedulerPlanFollower {
+
+  final static int GB = 1024;
+
+  private Clock mClock = null;
+  private CapacityScheduler scheduler = null;
+  private RMContext rmContext;
+  private RMContext spyRMContext;
+  private CapacitySchedulerContext csContext;
+  private ReservationAgent mAgent;
+  private Plan plan;
+  private Resource minAlloc = Resource.newInstance(GB, 1);
+  private Resource maxAlloc = Resource.newInstance(GB * 8, 8);
+  private ResourceCalculator res = new DefaultResourceCalculator();
+  private CapacityOverTimePolicy policy = new CapacityOverTimePolicy();
+
+  @Rule
+  public TestName name = new TestName();
+
+  @Before
+  public void setUp() throws Exception {
+    CapacityScheduler spyCs = new CapacityScheduler();
+    scheduler = spy(spyCs);
+    rmContext = TestUtils.getMockRMContext();
+    spyRMContext = spy(rmContext);
+
+    ConcurrentMap<ApplicationId, RMApp> spyApps =
+        spy(new ConcurrentHashMap<ApplicationId, RMApp>());
+    RMApp rmApp = mock(RMApp.class);
+    when(rmApp.getRMAppAttempt((ApplicationAttemptId) Matchers.any()))
+        .thenReturn(null);
+    Mockito.doReturn(rmApp).when(spyApps).get((ApplicationId) Matchers.any());
+    when(spyRMContext.getRMApps()).thenReturn(spyApps);
+
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration();
+    ReservationSystemTestUtil.setupQueueConfiguration(csConf);
+
+    scheduler.setConf(csConf);
+
+    csContext = mock(CapacitySchedulerContext.class);
+    when(csContext.getConfiguration()).thenReturn(csConf);
+    when(csContext.getConf()).thenReturn(csConf);
+    when(csContext.getMinimumResourceCapability()).thenReturn(minAlloc);
+    when(csContext.getMaximumResourceCapability()).thenReturn(maxAlloc);
+    when(csContext.getClusterResource()).thenReturn(
+        Resources.createResource(100 * 16 * GB, 100 * 32));
+    when(scheduler.getClusterResource()).thenReturn(
+        Resources.createResource(125 * GB, 125));
+    when(csContext.getResourceCalculator()).thenReturn(
+        new DefaultResourceCalculator());
+    RMContainerTokenSecretManager containerTokenSecretManager =
+        new RMContainerTokenSecretManager(csConf);
+    containerTokenSecretManager.rollMasterKey();
+    when(csContext.getContainerTokenSecretManager()).thenReturn(
+        containerTokenSecretManager);
+
+    scheduler.setRMContext(spyRMContext);
+    scheduler.init(csConf);
+    scheduler.start();
+
+    setupPlanFollower();
+  }
+
+  private void setupPlanFollower() throws Exception {
+    ReservationSystemTestUtil testUtil = new ReservationSystemTestUtil();
+    mClock = mock(Clock.class);
+    mAgent = mock(ReservationAgent.class);
+
+    String reservationQ = testUtil.getFullReservationQueueName();
+    CapacitySchedulerConfiguration csConf = scheduler.getConfiguration();
+    csConf.setReservationWindow(reservationQ, 20L);
+    csConf.setMaximumCapacity(reservationQ, 40);
+    csConf.setAverageCapacity(reservationQ, 20);
+    policy.init(reservationQ, csConf);
+  }
+
+  @Test
+  public void testWithMoveOnExpiry() throws PlanningException,
+      InterruptedException, AccessControlException {
+    // invoke plan follower test with move
+    testPlanFollower(true);
+  }
+
+  @Test
+  public void testWithKillOnExpiry() throws PlanningException,
+      InterruptedException, AccessControlException {
+    // invoke plan follower test with kill
+    testPlanFollower(false);
+  }
+
+  private void testPlanFollower(boolean isMove) throws PlanningException,
+      InterruptedException, AccessControlException {
+    // Initialize plan based on move flag
+    plan =
+        new InMemoryPlan(scheduler.getRootQueueMetrics(), policy, mAgent,
+            scheduler.getClusterResource(), 1L, res,
+            scheduler.getMinimumResourceCapability(), maxAlloc, "dedicated",
+            null, isMove);
+
+    // add a few reservations to the plan
+    long ts = System.currentTimeMillis();
+    ReservationId r1 = ReservationId.newInstance(ts, 1);
+    int[] f1 = { 10, 10, 10, 10, 10 };
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(r1, null, "u3",
+            "dedicated", 0, 0 + f1.length, ReservationSystemTestUtil
+                .generateAllocation(0L, 1L, f1), res, minAlloc)));
+
+    ReservationId r2 = ReservationId.newInstance(ts, 2);
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(r2, null, "u3",
+            "dedicated", 3, 3 + f1.length, ReservationSystemTestUtil
+                .generateAllocation(3L, 1L, f1), res, minAlloc)));
+
+    ReservationId r3 = ReservationId.newInstance(ts, 3);
+    int[] f2 = { 0, 10, 20, 10, 0 };
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(r3, null, "u4",
+            "dedicated", 10, 10 + f2.length, ReservationSystemTestUtil
+                .generateAllocation(10L, 1L, f2), res, minAlloc)));
+
+    CapacitySchedulerPlanFollower planFollower =
+        new CapacitySchedulerPlanFollower();
+    planFollower.init(mClock, scheduler, Collections.singletonList(plan));
+
+    when(mClock.getTime()).thenReturn(0L);
+    planFollower.run();
+
+    CSQueue defQ =
+        scheduler.getQueue("dedicated" + PlanQueue.DEFAULT_QUEUE_SUFFIX);
+    CSQueue q = scheduler.getQueue(r1.toString());
+    assertNotNull(q);
+    // submit an app to r1
+    String user_0 = "test-user";
+    ApplicationId appId = ApplicationId.newInstance(0, 1);
+    ApplicationAttemptId appAttemptId_0 =
+        ApplicationAttemptId.newInstance(appId, 0);
+    AppAddedSchedulerEvent addAppEvent =
+        new AppAddedSchedulerEvent(appId, q.getQueueName(), user_0);
+    scheduler.handle(addAppEvent);
+    AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
+        new AppAttemptAddedSchedulerEvent(appAttemptId_0, false);
+    scheduler.handle(appAttemptAddedEvent);
+
+    // initial default reservation queue should have no apps
+    Assert.assertEquals(0, defQ.getNumApplications());
+
+    Assert.assertEquals(0.1, q.getCapacity(), 0.01);
+    Assert.assertEquals(0.1, q.getMaximumCapacity(), 1.0);
+    Assert.assertEquals(1, q.getNumApplications());
+
+    CSQueue q2 = scheduler.getQueue(r2.toString());
+    assertNull(q2);
+    CSQueue q3 = scheduler.getQueue(r3.toString());
+    assertNull(q3);
+
+    when(mClock.getTime()).thenReturn(3L);
+    planFollower.run();
+
+    Assert.assertEquals(0, defQ.getNumApplications());
+    q = scheduler.getQueue(r1.toString());
+    assertNotNull(q);
+    Assert.assertEquals(0.1, q.getCapacity(), 0.01);
+    Assert.assertEquals(0.1, q.getMaximumCapacity(), 1.0);
+    Assert.assertEquals(1, q.getNumApplications());
+    q2 = scheduler.getQueue(r2.toString());
+    assertNotNull(q2);
+    Assert.assertEquals(0.1, q.getCapacity(), 0.01);
+    Assert.assertEquals(0.1, q.getMaximumCapacity(), 1.0);
+    q3 = scheduler.getQueue(r3.toString());
+    assertNull(q3);
+
+    when(mClock.getTime()).thenReturn(10L);
+    planFollower.run();
+
+    q = scheduler.getQueue(r1.toString());
+    if (isMove) {
+      // app should have been moved to default reservation queue
+      Assert.assertEquals(1, defQ.getNumApplications());
+      assertNull(q);
+    } else {
+      // app should be killed
+      Assert.assertEquals(0, defQ.getNumApplications());
+      assertNotNull(q);
+      AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent =
+          new AppAttemptRemovedSchedulerEvent(appAttemptId_0,
+              RMAppAttemptState.KILLED, false);
+      scheduler.handle(appAttemptRemovedEvent);
+    }
+    q2 = scheduler.getQueue(r2.toString());
+    assertNull(q2);
+    q3 = scheduler.getQueue(r3.toString());
+    assertNotNull(q3);
+    Assert.assertEquals(0, q3.getCapacity(), 0.01);
+    Assert.assertEquals(1.0, q3.getMaximumCapacity(), 1.0);
+
+    when(mClock.getTime()).thenReturn(11L);
+    planFollower.run();
+
+    if (isMove) {
+      // app should have been moved to default reservation queue
+      Assert.assertEquals(1, defQ.getNumApplications());
+    } else {
+      // app should be killed
+      Assert.assertEquals(0, defQ.getNumApplications());
+    }
+    q = scheduler.getQueue(r1.toString());
+    assertNull(q);
+    q2 = scheduler.getQueue(r2.toString());
+    assertNull(q2);
+    q3 = scheduler.getQueue(r3.toString());
+    assertNotNull(q3);
+    Assert.assertEquals(0.1, q3.getCapacity(), 0.01);
+    Assert.assertEquals(0.1, q3.getMaximumCapacity(), 1.0);
+
+    when(mClock.getTime()).thenReturn(12L);
+    planFollower.run();
+
+    q = scheduler.getQueue(r1.toString());
+    assertNull(q);
+    q2 = scheduler.getQueue(r2.toString());
+    assertNull(q2);
+    q3 = scheduler.getQueue(r3.toString());
+    assertNotNull(q3);
+    Assert.assertEquals(0.2, q3.getCapacity(), 0.01);
+    Assert.assertEquals(0.2, q3.getMaximumCapacity(), 1.0);
+
+    when(mClock.getTime()).thenReturn(16L);
+    planFollower.run();
+
+    q = scheduler.getQueue(r1.toString());
+    assertNull(q);
+    q2 = scheduler.getQueue(r2.toString());
+    assertNull(q2);
+    q3 = scheduler.getQueue(r3.toString());
+    assertNull(q3);
+
+    assertTrue(defQ.getCapacity() > 0.9);
+
+  }
+
+  public static ApplicationACLsManager mockAppACLsManager() {
+    Configuration conf = new Configuration();
+    return new ApplicationACLsManager(conf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (scheduler != null) {
+      scheduler.stop();
+    }
+  }
+
+}


[06/16] git commit: YARN-1709. In-memory data structures used to track resources over time to enable reservations. (cherry picked from commit 0d8b2cd88b958b1e602fd4ea4078ef8d4742a7c3)

Posted by cd...@apache.org.
YARN-1709. In-memory data structures used to track resources over time to enable reservations.
(cherry picked from commit 0d8b2cd88b958b1e602fd4ea4078ef8d4742a7c3)


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

Branch: refs/heads/trunk
Commit: cf4b34282aafee9f6b09d3433c4de1ae4b359168
Parents: c9266df
Author: subru <su...@outlook.com>
Authored: Fri Sep 12 17:22:08 2014 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Oct 3 15:41:51 2014 -0700

----------------------------------------------------------------------
 YARN-1051-CHANGES.txt                           |   3 +
 .../reservation/InMemoryPlan.java               | 507 +++++++++++++++++++
 .../InMemoryReservationAllocation.java          | 151 ++++++
 .../resourcemanager/reservation/Plan.java       |  32 ++
 .../reservation/PlanContext.java                | 101 ++++
 .../resourcemanager/reservation/PlanEdit.java   |  61 +++
 .../resourcemanager/reservation/PlanView.java   |  89 ++++
 .../RLESparseResourceAllocation.java            | 332 ++++++++++++
 .../reservation/ReservationAllocation.java      | 104 ++++
 .../reservation/ReservationInterval.java        |  67 +++
 .../exceptions/PlanningException.java           |  25 +
 .../reservation/ReservationSystemTestUtil.java  | 210 ++++++++
 .../reservation/TestInMemoryPlan.java           | 477 +++++++++++++++++
 .../TestInMemoryReservationAllocation.java      | 206 ++++++++
 .../TestRLESparseResourceAllocation.java        | 169 +++++++
 15 files changed, 2534 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/YARN-1051-CHANGES.txt
----------------------------------------------------------------------
diff --git a/YARN-1051-CHANGES.txt b/YARN-1051-CHANGES.txt
index a7c08a0..410d974 100644
--- a/YARN-1051-CHANGES.txt
+++ b/YARN-1051-CHANGES.txt
@@ -5,3 +5,6 @@ YARN-2475. Logic for responding to capacity drops for the
 ReservationSystem. (Carlo Curino and Subru Krishnan via curino)
 
 YARN-1708. Public YARN APIs for creating/updating/deleting reservations. (subru)
+
+YARN-1709. In-memory data structures used to track resources over time to
+enable reservations. (subru)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/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
new file mode 100644
index 0000000..99231c4
--- /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/InMemoryPlan.java
@@ -0,0 +1,507 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+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;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.UTCClock;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class InMemoryPlan implements Plan {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InMemoryPlan.class);
+
+  private static final Resource ZERO_RESOURCE = Resource.newInstance(0, 0);
+
+  private TreeMap<ReservationInterval, Set<InMemoryReservationAllocation>> currentReservations =
+      new TreeMap<ReservationInterval, Set<InMemoryReservationAllocation>>();
+
+  private RLESparseResourceAllocation rleSparseVector;
+
+  private Map<String, RLESparseResourceAllocation> userResourceAlloc =
+      new HashMap<String, RLESparseResourceAllocation>();
+
+  private Map<ReservationId, InMemoryReservationAllocation> reservationTable =
+      new HashMap<ReservationId, InMemoryReservationAllocation>();
+
+  private final ReentrantReadWriteLock readWriteLock =
+      new ReentrantReadWriteLock();
+  private final Lock readLock = readWriteLock.readLock();
+  private final Lock writeLock = readWriteLock.writeLock();
+  private final SharingPolicy policy;
+  private final ReservationAgent agent;
+  private final long step;
+  private final ResourceCalculator resCalc;
+  private final Resource minAlloc, maxAlloc;
+  private final String queueName;
+  private final QueueMetrics queueMetrics;
+  private final Planner replanner;
+  private final boolean getMoveOnExpiry;
+  private final Clock clock;
+
+  private Resource totalCapacity;
+
+  InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
+      ReservationAgent agent, Resource totalCapacity, long step,
+      ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
+      String queueName, Planner replanner, boolean getMoveOnExpiry) {
+    this(queueMetrics, policy, agent, totalCapacity, step, resCalc, minAlloc,
+        maxAlloc, queueName, replanner, getMoveOnExpiry, new UTCClock());
+  }
+
+  InMemoryPlan(QueueMetrics queueMetrics, SharingPolicy policy,
+      ReservationAgent agent, Resource totalCapacity, long step,
+      ResourceCalculator resCalc, Resource minAlloc, Resource maxAlloc,
+      String queueName, Planner replanner, boolean getMoveOnExpiry, Clock clock) {
+    this.queueMetrics = queueMetrics;
+    this.policy = policy;
+    this.agent = agent;
+    this.step = step;
+    this.totalCapacity = totalCapacity;
+    this.resCalc = resCalc;
+    this.minAlloc = minAlloc;
+    this.maxAlloc = maxAlloc;
+    this.rleSparseVector = new RLESparseResourceAllocation(resCalc, minAlloc);
+    this.queueName = queueName;
+    this.replanner = replanner;
+    this.getMoveOnExpiry = getMoveOnExpiry;
+    this.clock = clock;
+  }
+
+  @Override
+  public QueueMetrics getQueueMetrics() {
+    return queueMetrics;
+  }
+
+  private void incrementAllocation(ReservationAllocation reservation) {
+    assert (readWriteLock.isWriteLockedByCurrentThread());
+    Map<ReservationInterval, ReservationRequest> allocationRequests =
+        reservation.getAllocationRequests();
+    // check if we have encountered the user earlier and if not add an entry
+    String user = reservation.getUser();
+    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
+    if (resAlloc == null) {
+      resAlloc = new RLESparseResourceAllocation(resCalc, minAlloc);
+      userResourceAlloc.put(user, resAlloc);
+    }
+    for (Map.Entry<ReservationInterval, ReservationRequest> r : allocationRequests
+        .entrySet()) {
+      resAlloc.addInterval(r.getKey(), r.getValue());
+      rleSparseVector.addInterval(r.getKey(), r.getValue());
+    }
+  }
+
+  private void decrementAllocation(ReservationAllocation reservation) {
+    assert (readWriteLock.isWriteLockedByCurrentThread());
+    Map<ReservationInterval, ReservationRequest> allocationRequests =
+        reservation.getAllocationRequests();
+    String user = reservation.getUser();
+    RLESparseResourceAllocation resAlloc = userResourceAlloc.get(user);
+    for (Map.Entry<ReservationInterval, ReservationRequest> r : allocationRequests
+        .entrySet()) {
+      resAlloc.removeInterval(r.getKey(), r.getValue());
+      rleSparseVector.removeInterval(r.getKey(), r.getValue());
+    }
+    if (resAlloc.isEmpty()) {
+      userResourceAlloc.remove(resAlloc);
+    }
+  }
+
+  public Set<ReservationAllocation> getAllReservations() {
+    readLock.lock();
+    try {
+      if (currentReservations != null) {
+        Set<ReservationAllocation> flattenedReservations =
+            new HashSet<ReservationAllocation>();
+        for (Set<InMemoryReservationAllocation> reservationEntries : currentReservations
+            .values()) {
+          flattenedReservations.addAll(reservationEntries);
+        }
+        return flattenedReservations;
+      } else {
+        return null;
+      }
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public boolean addReservation(ReservationAllocation reservation)
+      throws PlanningException {
+    // Verify the allocation is memory based otherwise it is not supported
+    InMemoryReservationAllocation inMemReservation =
+        (InMemoryReservationAllocation) reservation;
+    if (inMemReservation.getUser() == null) {
+      String errMsg =
+          "The specified Reservation with ID "
+              + inMemReservation.getReservationId()
+              + " is not mapped to any user";
+      LOG.error(errMsg);
+      throw new IllegalArgumentException(errMsg);
+    }
+    writeLock.lock();
+    try {
+      if (reservationTable.containsKey(inMemReservation.getReservationId())) {
+        String errMsg =
+            "The specified Reservation with ID "
+                + inMemReservation.getReservationId() + " already exists";
+        LOG.error(errMsg);
+        throw new IllegalArgumentException(errMsg);
+      }
+      // Validate if we can accept this reservation, throws exception if
+      // validation fails
+      policy.validate(this, inMemReservation);
+      // we record here the time in which the allocation has been accepted
+      reservation.setAcceptanceTimestamp(clock.getTime());
+      ReservationInterval searchInterval =
+          new ReservationInterval(inMemReservation.getStartTime(),
+              inMemReservation.getEndTime());
+      Set<InMemoryReservationAllocation> reservations =
+          currentReservations.get(searchInterval);
+      if (reservations == null) {
+        reservations = new HashSet<InMemoryReservationAllocation>();
+      }
+      if (!reservations.add(inMemReservation)) {
+        LOG.error("Unable to add reservation: {} to plan.",
+            inMemReservation.getReservationId());
+        return false;
+      }
+      currentReservations.put(searchInterval, reservations);
+      reservationTable.put(inMemReservation.getReservationId(),
+          inMemReservation);
+      incrementAllocation(inMemReservation);
+      LOG.info("Sucessfully added reservation: {} to plan.",
+          inMemReservation.getReservationId());
+      return true;
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  @Override
+  public boolean updateReservation(ReservationAllocation reservation)
+      throws PlanningException {
+    writeLock.lock();
+    boolean result = false;
+    try {
+      ReservationId resId = reservation.getReservationId();
+      ReservationAllocation currReservation = getReservationById(resId);
+      if (currReservation == null) {
+        String errMsg =
+            "The specified Reservation with ID " + resId
+                + " does not exist in the plan";
+        LOG.error(errMsg);
+        throw new IllegalArgumentException(errMsg);
+      }
+      if (!removeReservation(currReservation)) {
+        LOG.error("Unable to replace reservation: {} from plan.",
+            reservation.getReservationId());
+        return result;
+      }
+      try {
+        result = addReservation(reservation);
+      } catch (PlanningException e) {
+        LOG.error("Unable to update reservation: {} from plan due to {}.",
+            reservation.getReservationId(), e.getMessage());
+      }
+      if (result) {
+        LOG.info("Sucessfully updated reservation: {} in plan.",
+            reservation.getReservationId());
+        return result;
+      } else {
+        // rollback delete
+        addReservation(currReservation);
+        LOG.info("Rollbacked update reservation: {} from plan.",
+            reservation.getReservationId());
+        return result;
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private boolean removeReservation(ReservationAllocation reservation) {
+    assert (readWriteLock.isWriteLockedByCurrentThread());
+    ReservationInterval searchInterval =
+        new ReservationInterval(reservation.getStartTime(),
+            reservation.getEndTime());
+    Set<InMemoryReservationAllocation> reservations =
+        currentReservations.get(searchInterval);
+    if (reservations != null) {
+      if (!reservations.remove(reservation)) {
+        LOG.error("Unable to remove reservation: {} from plan.",
+            reservation.getReservationId());
+        return false;
+      }
+      if (reservations.isEmpty()) {
+        currentReservations.remove(searchInterval);
+      }
+    } else {
+      String errMsg =
+          "The specified Reservation with ID " + reservation.getReservationId()
+              + " does not exist in the plan";
+      LOG.error(errMsg);
+      throw new IllegalArgumentException(errMsg);
+    }
+    reservationTable.remove(reservation.getReservationId());
+    decrementAllocation(reservation);
+    LOG.info("Sucessfully deleted reservation: {} in plan.",
+        reservation.getReservationId());
+    return true;
+  }
+
+  @Override
+  public boolean deleteReservation(ReservationId reservationID) {
+    writeLock.lock();
+    try {
+      ReservationAllocation reservation = getReservationById(reservationID);
+      if (reservation == null) {
+        String errMsg =
+            "The specified Reservation with ID " + reservationID
+                + " does not exist in the plan";
+        LOG.error(errMsg);
+        throw new IllegalArgumentException(errMsg);
+      }
+      return removeReservation(reservation);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  @Override
+  public void archiveCompletedReservations(long tick) {
+    // Since we are looking for old reservations, read lock is optimal
+    LOG.debug("Running archival at time: {}", tick);
+    readLock.lock();
+    List<InMemoryReservationAllocation> expiredReservations =
+        new ArrayList<InMemoryReservationAllocation>();
+    // archive reservations and delete the ones which are beyond
+    // the reservation policy "window"
+    try {
+      long archivalTime = tick - policy.getValidWindow();
+      ReservationInterval searchInterval =
+          new ReservationInterval(archivalTime, archivalTime);
+      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>> reservations =
+          currentReservations.headMap(searchInterval, true);
+      if (!reservations.isEmpty()) {
+        for (Set<InMemoryReservationAllocation> reservationEntries : reservations
+            .values()) {
+          for (InMemoryReservationAllocation reservation : reservationEntries) {
+            if (reservation.getEndTime() <= archivalTime) {
+              expiredReservations.add(reservation);
+            }
+          }
+        }
+      }
+    } finally {
+      readLock.unlock();
+    }
+    if (expiredReservations.isEmpty()) {
+      return;
+    }
+    // Need write lock only if there are any reservations to be deleted
+    writeLock.lock();
+    try {
+      for (InMemoryReservationAllocation expiredReservation : expiredReservations) {
+        removeReservation(expiredReservation);
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  @Override
+  public Set<ReservationAllocation> getReservationsAtTime(long tick) {
+    readLock.lock();
+    ReservationInterval searchInterval =
+        new ReservationInterval(tick, Long.MAX_VALUE);
+    try {
+      SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>> reservations =
+          currentReservations.headMap(searchInterval, true);
+      if (!reservations.isEmpty()) {
+        Set<ReservationAllocation> flattenedReservations =
+            new HashSet<ReservationAllocation>();
+        for (Set<InMemoryReservationAllocation> reservationEntries : reservations
+            .values()) {
+          for (InMemoryReservationAllocation reservation : reservationEntries) {
+            if (reservation.getEndTime() > tick) {
+              flattenedReservations.add(reservation);
+            }
+          }
+        }
+        return Collections.unmodifiableSet(flattenedReservations);
+      } else {
+        return Collections.emptySet();
+      }
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public long getStep() {
+    return step;
+  }
+
+  @Override
+  public SharingPolicy getSharingPolicy() {
+    return policy;
+  }
+
+  @Override
+  public ReservationAgent getReservationAgent() {
+    return agent;
+  }
+
+  @Override
+  public Resource getConsumptionForUser(String user, long t) {
+    readLock.lock();
+    try {
+      RLESparseResourceAllocation userResAlloc = userResourceAlloc.get(user);
+      if (userResAlloc != null) {
+        return userResAlloc.getCapacityAtTime(t);
+      } else {
+        return Resources.clone(ZERO_RESOURCE);
+      }
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public Resource getTotalCommittedResources(long t) {
+    readLock.lock();
+    try {
+      return rleSparseVector.getCapacityAtTime(t);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public ReservationAllocation getReservationById(ReservationId reservationID) {
+    if (reservationID == null) {
+      return null;
+    }
+    readLock.lock();
+    try {
+      return reservationTable.get(reservationID);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public Resource getTotalCapacity() {
+    readLock.lock();
+    try {
+      return Resources.clone(totalCapacity);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public Resource getMinimumAllocation() {
+    return Resources.clone(minAlloc);
+  }
+
+  @Override
+  public void setTotalCapacity(Resource cap) {
+    writeLock.lock();
+    try {
+      totalCapacity = Resources.clone(cap);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public long getEarliestStartTime() {
+    readLock.lock();
+    try {
+      return rleSparseVector.getEarliestStartTime();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public long getLastEndTime() {
+    readLock.lock();
+    try {
+      return rleSparseVector.getLatestEndTime();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public ResourceCalculator getResourceCalculator() {
+    return resCalc;
+  }
+
+  @Override
+  public String getQueueName() {
+    return queueName;
+  }
+
+  @Override
+  public Resource getMaximumAllocation() {
+    return Resources.clone(maxAlloc);
+  }
+
+  public String toCumulativeString() {
+    readLock.lock();
+    try {
+      return rleSparseVector.toString();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public Planner getReplanner() {
+    return replanner;
+  }
+
+  @Override
+  public boolean getMoveOnExpiry() {
+    return getMoveOnExpiry;
+  }
+
+  @Override
+  public String toString() {
+    readLock.lock();
+    try {
+      StringBuffer planStr = new StringBuffer("In-memory Plan: ");
+      planStr.append("Parent Queue: ").append(queueName)
+          .append("Total Capacity: ").append(totalCapacity).append("Step: ")
+          .append(step);
+      for (ReservationAllocation reservation : getAllReservations()) {
+        planStr.append(reservation);
+      }
+      return planStr.toString();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/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
new file mode 100644
index 0000000..10cc55f
--- /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/InMemoryReservationAllocation.java
@@ -0,0 +1,151 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import java.util.Collections;
+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;
+
+/**
+ * An in memory implementation of a reservation allocation using the
+ * {@link RLESparseResourceAllocation}
+ * 
+ */
+class InMemoryReservationAllocation implements ReservationAllocation {
+
+  private final String planName;
+  private final ReservationId reservationID;
+  private final String user;
+  private final ReservationDefinition contract;
+  private final long startTime;
+  private final long endTime;
+  private final Map<ReservationInterval, ReservationRequest> allocationRequests;
+  private boolean hasGang = false;
+  private long acceptedAt = -1;
+
+  private RLESparseResourceAllocation resourcesOverTime;
+
+  InMemoryReservationAllocation(ReservationId reservationID,
+      ReservationDefinition contract, String user, String planName,
+      long startTime, long endTime,
+      Map<ReservationInterval, ReservationRequest> allocationRequests,
+      ResourceCalculator calculator, Resource minAlloc) {
+    this.contract = contract;
+    this.startTime = startTime;
+    this.endTime = endTime;
+    this.reservationID = reservationID;
+    this.user = user;
+    this.allocationRequests = allocationRequests;
+    this.planName = planName;
+    resourcesOverTime = new RLESparseResourceAllocation(calculator, minAlloc);
+    for (Map.Entry<ReservationInterval, ReservationRequest> r : allocationRequests
+        .entrySet()) {
+      resourcesOverTime.addInterval(r.getKey(), r.getValue());
+      if (r.getValue().getConcurrency() > 1) {
+        hasGang = true;
+      }
+    }
+  }
+
+  @Override
+  public ReservationId getReservationId() {
+    return reservationID;
+  }
+
+  @Override
+  public ReservationDefinition getReservationDefinition() {
+    return contract;
+  }
+
+  @Override
+  public long getStartTime() {
+    return startTime;
+  }
+
+  @Override
+  public long getEndTime() {
+    return endTime;
+  }
+
+  @Override
+  public Map<ReservationInterval, ReservationRequest> getAllocationRequests() {
+    return Collections.unmodifiableMap(allocationRequests);
+  }
+
+  @Override
+  public String getPlanName() {
+    return planName;
+  }
+
+  @Override
+  public String getUser() {
+    return user;
+  }
+
+  @Override
+  public boolean containsGangs() {
+    return hasGang;
+  }
+
+  @Override
+  public void setAcceptanceTimestamp(long acceptedAt) {
+    this.acceptedAt = acceptedAt;
+  }
+
+  @Override
+  public long getAcceptanceTime() {
+    return acceptedAt;
+  }
+
+  @Override
+  public Resource getResourcesAtTime(long tick) {
+    if (tick < startTime || tick >= endTime) {
+      return Resource.newInstance(0, 0);
+    }
+    return Resources.clone(resourcesOverTime.getCapacityAtTime(tick));
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sBuf = new StringBuilder();
+    sBuf.append(getReservationId()).append(" user:").append(getUser())
+        .append(" startTime: ").append(getStartTime()).append(" endTime: ")
+        .append(getEndTime()).append(" alloc:[")
+        .append(resourcesOverTime.toString()).append("] ");
+    return sBuf.toString();
+  }
+
+  @Override
+  public int compareTo(ReservationAllocation other) {
+    // reverse order of acceptance
+    if (this.getAcceptanceTime() > other.getAcceptanceTime()) {
+      return -1;
+    }
+    if (this.getAcceptanceTime() < other.getAcceptanceTime()) {
+      return 1;
+    }
+    return 0;
+  }
+
+  @Override
+  public int hashCode() {
+    return reservationID.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    InMemoryReservationAllocation other = (InMemoryReservationAllocation) obj;
+    return this.reservationID.equals(other.getReservationId());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/Plan.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/Plan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/Plan.java
new file mode 100644
index 0000000..cf2aed7
--- /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/Plan.java
@@ -0,0 +1,32 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+/**
+ * A Plan represents the central data structure of a reservation system that
+ * maintains the "agenda" for the cluster. In particular, it maintains
+ * information on how a set of {@link ReservationDefinition} that have been
+ * previously accepted will be honored.
+ * 
+ * {@link ReservationDefinition} submitted by the users through the RM public
+ * APIs are passed to appropriate {@link ReservationAgent}s, which in turn will
+ * consult the Plan (via the {@link PlanView} interface) and try to determine
+ * whether there are sufficient resources available in this Plan to satisfy the
+ * temporal and resource constraints of a {@link ReservationDefinition}. If a
+ * valid allocation is found the agent will try to store it in the plan (via the
+ * {@link PlanEdit} interface). Upon success the system return to the user a
+ * positive acknowledgment, and a reservation identifier to be later used to
+ * access the reserved resources.
+ * 
+ * A {@link PlanFollower} will continuously read from the Plan and will
+ * affect the instantaneous allocation of resources among jobs running by
+ * publishing the "current" slice of the Plan to the underlying scheduler. I.e.,
+ * the configuration of queues/weights of the scheduler are modified to reflect
+ * the allocations in the Plan.
+ * 
+ * As this interface have several methods we decompose them into three groups:
+ * {@link PlanContext}: containing configuration type information,
+ * {@link PlanView} read-only access to the plan state, and {@link PlanEdit}
+ * write access to the plan state.
+ */
+public interface Plan extends PlanContext, PlanView, PlanEdit {
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanContext.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/PlanContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanContext.java
new file mode 100644
index 0000000..40a25a6
--- /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/PlanContext.java
@@ -0,0 +1,101 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+
+/**
+ * This interface provides read-only access to configuration-type parameter for
+ * a plan.
+ * 
+ */
+public interface PlanContext {
+
+  /**
+   * Returns the configured "step" or granularity of time of the plan in millis.
+   * 
+   * @return plan step in millis
+   */
+  public long getStep();
+
+  /**
+   * Return the {@link ReservationAgent} configured for this plan that is
+   * responsible for optimally placing various reservation requests
+   * 
+   * @return the {@link ReservationAgent} configured for this plan
+   */
+  public ReservationAgent getReservationAgent();
+
+  /**
+   * Return an instance of a {@link Planner}, which will be invoked in response
+   * to unexpected reduction in the resources of this plan
+   * 
+   * @return an instance of a {@link Planner}, which will be invoked in response
+   *         to unexpected reduction in the resources of this plan
+   */
+  public Planner getReplanner();
+
+  /**
+   * Return the configured {@link SharingPolicy} that governs the sharing of the
+   * resources of the plan between its various users
+   * 
+   * @return the configured {@link SharingPolicy} that governs the sharing of
+   *         the resources of the plan between its various users
+   */
+  public SharingPolicy getSharingPolicy();
+
+  /**
+   * Returns the system {@link ResourceCalculator}
+   * 
+   * @return the system {@link ResourceCalculator}
+   */
+  public ResourceCalculator getResourceCalculator();
+
+  /**
+   * Returns the single smallest {@link Resource} allocation that can be
+   * reserved in this plan
+   * 
+   * @return the single smallest {@link Resource} allocation that can be
+   *         reserved in this plan
+   */
+  public Resource getMinimumAllocation();
+
+  /**
+   * Returns the single largest {@link Resource} allocation that can be reserved
+   * in this plan
+   * 
+   * @return the single largest {@link Resource} allocation that can be reserved
+   *         in this plan
+   */
+  public Resource getMaximumAllocation();
+
+  /**
+   * Return the name of the queue in the {@link ResourceScheduler} corresponding
+   * to this plan
+   * 
+   * @return the name of the queue in the {@link ResourceScheduler}
+   *         corresponding to this plan
+   */
+  public String getQueueName();
+
+  /**
+   * Return the {@link QueueMetrics} for the queue in the
+   * {@link ResourceScheduler} corresponding to this plan
+   * 
+   * @return the {@link QueueMetrics} for the queue in the
+   *         {@link ResourceScheduler} corresponding to this plan
+   */
+  public QueueMetrics getQueueMetrics();
+
+  /**
+   * Instructs the {@link PlanFollower} on what to do for applications
+   * which are still running when the reservation is expiring (move-to-default
+   * vs kill)
+   * 
+   * @return true if remaining applications have to be killed, false if they
+   *         have to migrated
+   */
+  public boolean getMoveOnExpiry();
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.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/PlanEdit.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanEdit.java
new file mode 100644
index 0000000..648edba
--- /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/PlanEdit.java
@@ -0,0 +1,61 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+
+/**
+ * This interface groups the methods used to modify the state of a Plan.
+ */
+public interface PlanEdit extends PlanContext, PlanView {
+
+  /**
+   * Add a new {@link ReservationAllocation} to the plan
+   * 
+   * @param reservation the {@link ReservationAllocation} to be added to the
+   *          plan
+   * @return true if addition is successful, false otherwise
+   */
+  public boolean addReservation(ReservationAllocation reservation)
+      throws PlanningException;
+
+  /**
+   * Updates an existing {@link ReservationAllocation} in the plan. This is
+   * required for re-negotiation
+   * 
+   * @param reservation the {@link ReservationAllocation} to be updated the plan
+   * @return true if update is successful, false otherwise
+   */
+  public boolean updateReservation(ReservationAllocation reservation)
+      throws PlanningException;
+
+  /**
+   * Delete an existing {@link ReservationAllocation} from the plan identified
+   * uniquely by its {@link ReservationId}. This will generally be used for
+   * garbage collection
+   * 
+   * @param reservation the {@link ReservationAllocation} to be deleted from the
+   *          plan identified uniquely by its {@link ReservationId}
+   * @return true if delete is successful, false otherwise
+   */
+  public boolean deleteReservation(ReservationId reservationID)
+      throws PlanningException;
+
+  /**
+   * Method invoked to garbage collect old reservations. It cleans up expired
+   * reservations that have fallen out of the sliding archival window
+   * 
+   * @param tick the current time from which the archival window is computed
+   */
+  public void archiveCompletedReservations(long tick) throws PlanningException;
+
+  /**
+   * Sets the overall capacity in terms of {@link Resource} assigned to this
+   * plan
+   * 
+   * @param capacity the overall capacity in terms of {@link Resource} assigned
+   *          to this plan
+   */
+  public void setTotalCapacity(Resource capacity);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.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/PlanView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PlanView.java
new file mode 100644
index 0000000..6e58dde
--- /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/PlanView.java
@@ -0,0 +1,89 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+/**
+ * This interface provides a read-only view on the allocations made in this
+ * plan. This methods are used for example by {@link ReservationAgent}s to
+ * determine the free resources in a certain point in time, and by
+ * PlanFollowerPolicy to publish this plan to the scheduler.
+ */
+public interface PlanView extends PlanContext {
+
+  /**
+   * Return a {@link ReservationAllocation} identified by its
+   * {@link ReservationId}
+   * 
+   * @param reservationID the unique id to identify the
+   *          {@link ReservationAllocation}
+   * @return {@link ReservationAllocation} identified by the specified id
+   */
+  public ReservationAllocation getReservationById(ReservationId reservationID);
+
+  /**
+   * Gets all the active reservations at the specified point of time
+   * 
+   * @param tick the time (UTC in ms) for which the active reservations are
+   *          requested
+   * @return set of active reservations at the specified time
+   */
+  public Set<ReservationAllocation> getReservationsAtTime(long tick);
+
+  /**
+   * Gets all the reservations in the plan
+   * 
+   * @return set of all reservations handled by this Plan
+   */
+  public Set<ReservationAllocation> getAllReservations();
+
+  /**
+   * Returns the total {@link Resource} reserved for all users at the specified
+   * time
+   * 
+   * @param tick the time (UTC in ms) for which the reserved resources are
+   *          requested
+   * @return the total {@link Resource} reserved for all users at the specified
+   *         time
+   */
+  public Resource getTotalCommittedResources(long tick);
+
+  /**
+   * Returns the total {@link Resource} reserved for a given user at the
+   * specified time
+   * 
+   * @param user the user who made the reservation(s)
+   * @param tick the time (UTC in ms) for which the reserved resources are
+   *          requested
+   * @return the total {@link Resource} reserved for a given user at the
+   *         specified time
+   */
+  public Resource getConsumptionForUser(String user, long tick);
+
+  /**
+   * Returns the overall capacity in terms of {@link Resource} assigned to this
+   * plan (typically will correspond to the absolute capacity of the
+   * corresponding queue).
+   * 
+   * @return the overall capacity in terms of {@link Resource} assigned to this
+   *         plan
+   */
+  public Resource getTotalCapacity();
+
+  /**
+   * Gets the time (UTC in ms) at which the first reservation starts
+   * 
+   * @return the time (UTC in ms) at which the first reservation starts
+   */
+  public long getEarliestStartTime();
+
+  /**
+   * Returns the time (UTC in ms) at which the last reservation terminates
+   * 
+   * @return the time (UTC in ms) at which the last reservation terminates
+   */
+  public long getLastEndTime();
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/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
new file mode 100644
index 0000000..fa8db30
--- /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/RLESparseResourceAllocation.java
@@ -0,0 +1,332 @@
+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;
+import java.util.Set;
+import java.util.SortedMap;
+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;
+
+import com.google.gson.stream.JsonWriter;
+
+/**
+ * This is a run length encoded sparse data structure that maintains resource
+ * allocations over time
+ */
+public class RLESparseResourceAllocation {
+
+  private static final int THRESHOLD = 100;
+  private static final Resource ZERO_RESOURCE = Resource.newInstance(0, 0);
+
+  private TreeMap<Long, Resource> cumulativeCapacity =
+      new TreeMap<Long, Resource>();
+
+  private final ReentrantReadWriteLock readWriteLock =
+      new ReentrantReadWriteLock();
+  private final Lock readLock = readWriteLock.readLock();
+  private final Lock writeLock = readWriteLock.writeLock();
+
+  private final ResourceCalculator resourceCalculator;
+  private final Resource minAlloc;
+
+  public RLESparseResourceAllocation(ResourceCalculator resourceCalculator,
+      Resource minAlloc) {
+    this.resourceCalculator = resourceCalculator;
+    this.minAlloc = minAlloc;
+  }
+
+  private boolean isSameAsPrevious(Long key, Resource capacity) {
+    Entry<Long, Resource> previous = cumulativeCapacity.lowerEntry(key);
+    return (previous != null && previous.getValue().equals(capacity));
+  }
+
+  private boolean isSameAsNext(Long key, Resource capacity) {
+    Entry<Long, Resource> next = cumulativeCapacity.higherEntry(key);
+    return (next != null && next.getValue().equals(capacity));
+  }
+
+  /**
+   * Add a resource for the specified interval
+   * 
+   * @param reservationInterval the interval for which the resource is to be
+   *          added
+   * @param capacity 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());
+    if (totCap.equals(ZERO_RESOURCE)) {
+      return true;
+    }
+    writeLock.lock();
+    try {
+      long startKey = reservationInterval.getStartTime();
+      long endKey = reservationInterval.getEndTime();
+      NavigableMap<Long, Resource> ticks =
+          cumulativeCapacity.headMap(endKey, false);
+      if (ticks != null && !ticks.isEmpty()) {
+        Resource updatedCapacity = Resource.newInstance(0, 0);
+        Entry<Long, Resource> lowEntry = ticks.floorEntry(startKey);
+        if (lowEntry == null) {
+          // This is the earliest starting interval
+          cumulativeCapacity.put(startKey, totCap);
+        } else {
+          updatedCapacity = Resources.add(lowEntry.getValue(), totCap);
+          // Add a new tick only if the updated value is different
+          // from the previous tick
+          if ((startKey == lowEntry.getKey())
+              && (isSameAsPrevious(lowEntry.getKey(), updatedCapacity))) {
+            cumulativeCapacity.remove(lowEntry.getKey());
+          } else {
+            cumulativeCapacity.put(startKey, updatedCapacity);
+          }
+        }
+        // Increase all the capacities of overlapping intervals
+        Set<Entry<Long, Resource>> overlapSet =
+            ticks.tailMap(startKey, false).entrySet();
+        for (Entry<Long, Resource> entry : overlapSet) {
+          updatedCapacity = Resources.add(entry.getValue(), totCap);
+          entry.setValue(updatedCapacity);
+        }
+      } else {
+        // This is the first interval to be added
+        cumulativeCapacity.put(startKey, totCap);
+      }
+      Resource nextTick = cumulativeCapacity.get(endKey);
+      if (nextTick != null) {
+        // If there is overlap, remove the duplicate entry
+        if (isSameAsPrevious(endKey, nextTick)) {
+          cumulativeCapacity.remove(endKey);
+        }
+      } else {
+        // Decrease capacity as this is end of the interval
+        cumulativeCapacity.put(endKey, Resources.subtract(cumulativeCapacity
+            .floorEntry(endKey).getValue(), totCap));
+      }
+      return true;
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * 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
+   * @return true if removal is successful, false otherwise
+   */
+  public boolean removeInterval(ReservationInterval reservationInterval,
+      ReservationRequest capacity) {
+    Resource totCap =
+        Resources.multiply(capacity.getCapability(),
+            (float) capacity.getNumContainers());
+    if (totCap.equals(ZERO_RESOURCE)) {
+      return true;
+    }
+    writeLock.lock();
+    try {
+      long startKey = reservationInterval.getStartTime();
+      long endKey = reservationInterval.getEndTime();
+      // update the start key
+      NavigableMap<Long, Resource> ticks =
+          cumulativeCapacity.headMap(endKey, false);
+      // Decrease all the capacities of overlapping intervals
+      SortedMap<Long, Resource> overlapSet = ticks.tailMap(startKey);
+      if (overlapSet != null && !overlapSet.isEmpty()) {
+        Resource updatedCapacity = Resource.newInstance(0, 0);
+        long currentKey = -1;
+        for (Iterator<Entry<Long, Resource>> overlapEntries =
+            overlapSet.entrySet().iterator(); overlapEntries.hasNext();) {
+          Entry<Long, Resource> entry = overlapEntries.next();
+          currentKey = entry.getKey();
+          updatedCapacity = Resources.subtract(entry.getValue(), totCap);
+          // update each entry between start and end key
+          cumulativeCapacity.put(currentKey, updatedCapacity);
+        }
+        // Remove the first overlap entry if it is same as previous after
+        // updation
+        Long firstKey = overlapSet.firstKey();
+        if (isSameAsPrevious(firstKey, overlapSet.get(firstKey))) {
+          cumulativeCapacity.remove(firstKey);
+        }
+        // Remove the next entry if it is same as end entry after updation
+        if ((currentKey != -1) && (isSameAsNext(currentKey, updatedCapacity))) {
+          cumulativeCapacity.remove(cumulativeCapacity.higherKey(currentKey));
+        }
+      }
+      return true;
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Returns the capacity, i.e. total resources allocated at the specified point
+   * of time
+   * 
+   * @param tick the time (UTC in ms) at which the capacity is requested
+   * @return the resources allocated at the specified time
+   */
+  public Resource getCapacityAtTime(long tick) {
+    readLock.lock();
+    try {
+      Entry<Long, Resource> closestStep = cumulativeCapacity.floorEntry(tick);
+      if (closestStep != null) {
+        return Resources.clone(closestStep.getValue());
+      }
+      return Resources.clone(ZERO_RESOURCE);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Get the timestamp of the earliest resource allocation
+   * 
+   * @return the timestamp of the first resource allocation
+   */
+  public long getEarliestStartTime() {
+    readLock.lock();
+    try {
+      if (cumulativeCapacity.isEmpty()) {
+        return -1;
+      } else {
+        return cumulativeCapacity.firstKey();
+      }
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Get the timestamp of the latest resource allocation
+   * 
+   * @return the timestamp of the last resource allocation
+   */
+  public long getLatestEndTime() {
+    readLock.lock();
+    try {
+      if (cumulativeCapacity.isEmpty()) {
+        return -1;
+      } else {
+        return cumulativeCapacity.lastKey();
+      }
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Returns true if there are no non-zero entries
+   * 
+   * @return true if there are no allocations or false otherwise
+   */
+  public boolean isEmpty() {
+    readLock.lock();
+    try {
+      if (cumulativeCapacity.isEmpty()) {
+        return true;
+      }
+      // Deletion leaves a single zero entry so check for that
+      if (cumulativeCapacity.size() == 1) {
+        return cumulativeCapacity.firstEntry().getValue().equals(ZERO_RESOURCE);
+      }
+      return false;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder ret = new StringBuilder();
+    readLock.lock();
+    try {
+      if (cumulativeCapacity.size() > THRESHOLD) {
+        ret.append("Number of steps: ").append(cumulativeCapacity.size())
+            .append(" earliest entry: ").append(cumulativeCapacity.firstKey())
+            .append(" latest entry: ").append(cumulativeCapacity.lastKey());
+      } else {
+        for (Map.Entry<Long, Resource> r : cumulativeCapacity.entrySet()) {
+          ret.append(r.getKey()).append(": ").append(r.getValue())
+              .append("\n ");
+        }
+      }
+      return ret.toString();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Returns the JSON string representation of the current resources allocated
+   * over time
+   * 
+   * @return the JSON string representation of the current resources allocated
+   *         over time
+   */
+  public String toMemJSONString() {
+    StringWriter json = new StringWriter();
+    JsonWriter jsonWriter = new JsonWriter(json);
+    readLock.lock();
+    try {
+      jsonWriter.beginObject();
+      // jsonWriter.name("timestamp").value("resource");
+      for (Map.Entry<Long, Resource> r : cumulativeCapacity.entrySet()) {
+        jsonWriter.name(r.getKey().toString()).value(r.getValue().toString());
+      }
+      jsonWriter.endObject();
+      jsonWriter.close();
+      return json.toString();
+    } catch (IOException e) {
+      // This should not happen
+      return "";
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/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
new file mode 100644
index 0000000..bca3aa8
--- /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/ReservationAllocation.java
@@ -0,0 +1,104 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+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;
+
+/**
+ * A ReservationAllocation represents a concrete allocation of resources over
+ * time that satisfy a certain {@link ReservationDefinition}. This is used
+ * internally by a {@link Plan} to store information about how each of the
+ * accepted {@link ReservationDefinition} have been allocated.
+ */
+public interface ReservationAllocation extends
+    Comparable<ReservationAllocation> {
+
+  /**
+   * Returns the unique identifier {@link ReservationId} that represents the
+   * reservation
+   * 
+   * @return reservationId the unique identifier {@link ReservationId} that
+   *         represents the reservation
+   */
+  public ReservationId getReservationId();
+
+  /**
+   * Returns the original {@link ReservationDefinition} submitted by the client
+   * 
+   * @return
+   */
+  public ReservationDefinition getReservationDefinition();
+
+  /**
+   * Returns the time at which the reservation is activated
+   * 
+   * @return the time at which the reservation is activated
+   */
+  public long getStartTime();
+
+  /**
+   * Returns the time at which the reservation terminates
+   * 
+   * @return the time at which the reservation terminates
+   */
+  public long getEndTime();
+
+  /**
+   * Returns the map of resources requested against the time interval for which
+   * they were
+   * 
+   * @return the allocationRequests the map of resources requested against the
+   *         time interval for which they were
+   */
+  public Map<ReservationInterval, ReservationRequest> getAllocationRequests();
+
+  /**
+   * Return a string identifying the plan to which the reservation belongs
+   * 
+   * @return the plan to which the reservation belongs
+   */
+  public String getPlanName();
+
+  /**
+   * Returns the user who requested the reservation
+   * 
+   * @return the user who requested the reservation
+   */
+  public String getUser();
+
+  /**
+   * Returns whether the reservation has gang semantics or not
+   * 
+   * @return true if there is a gang request, false otherwise
+   */
+  public boolean containsGangs();
+
+  /**
+   * Sets the time at which the reservation was accepted by the system
+   * 
+   * @param acceptedAt the time at which the reservation was accepted by the
+   *          system
+   */
+  public void setAcceptanceTimestamp(long acceptedAt);
+
+  /**
+   * Returns the time at which the reservation was accepted by the system
+   * 
+   * @return the time at which the reservation was accepted by the system
+   */
+  public long getAcceptanceTime();
+
+  /**
+   * Returns the capacity represented by cumulative resources reserved by the
+   * reservation at the specified point of time
+   * 
+   * @param tick the time (UTC in ms) for which the reserved resources are
+   *          requested
+   * @return the resources reserved at the specified time
+   */
+  public Resource getResourcesAtTime(long tick);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInterval.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/ReservationInterval.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInterval.java
new file mode 100644
index 0000000..d3a6d51
--- /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/ReservationInterval.java
@@ -0,0 +1,67 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+/**
+ * This represents the time duration of the reservation
+ * 
+ */
+public class ReservationInterval implements Comparable<ReservationInterval> {
+
+  private final long startTime;
+
+  private final long endTime;
+
+  public ReservationInterval(long startTime, long endTime) {
+    this.startTime = startTime;
+    this.endTime = endTime;
+  }
+
+  /**
+   * Get the start time of the reservation interval
+   * 
+   * @return the startTime
+   */
+  public long getStartTime() {
+    return startTime;
+  }
+
+  /**
+   * Get the end time of the reservation interval
+   * 
+   * @return the endTime
+   */
+  public long getEndTime() {
+    return endTime;
+  }
+
+  /**
+   * Returns whether the interval is active at the specified instant of time
+   * 
+   * @param tick the instance of the time to check
+   * @return true if active, false otherwise
+   */
+  public boolean isOverlap(long tick) {
+    return (startTime <= tick && tick <= endTime);
+  }
+
+  @Override
+  public int compareTo(ReservationInterval anotherInterval) {
+    long diff = 0;
+    if (startTime == anotherInterval.getStartTime()) {
+      diff = endTime - anotherInterval.getEndTime();
+    } else {
+      diff = startTime - anotherInterval.getStartTime();
+    }
+    if (diff < 0) {
+      return -1;
+    } else if (diff > 0) {
+      return 1;
+    } else {
+      return 0;
+    }
+  }
+
+  public String toString() {
+    return "[" + startTime + ", " + endTime + "]";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningException.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/exceptions/PlanningException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/exceptions/PlanningException.java
new file mode 100644
index 0000000..aa9e9fb
--- /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/exceptions/PlanningException.java
@@ -0,0 +1,25 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+
+/**
+ * Exception thrown by the admission control subsystem when there is a problem
+ * in trying to find an allocation for a user {@link ReservationSubmissionRequest}.
+ */
+public class PlanningException extends Exception {
+
+  private static final long serialVersionUID = -684069387367879218L;
+
+  public PlanningException(String message) {
+    super(message);
+  }
+
+  public PlanningException(Throwable cause) {
+    super(cause);
+  }
+
+  public PlanningException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf4b3428/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
new file mode 100644
index 0000000..cbca6dc
--- /dev/null
+++ 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
@@ -0,0 +1,210 @@
+package org.apache.hadoop.yarn.server.resourcemanager.reservation;
+
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Random;
+import java.util.TreeMap;
+
+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.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationDefinitionPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationRequestsPBImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
+import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
+import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.junit.Assert;
+import org.mockito.Mockito;
+
+public class ReservationSystemTestUtil {
+
+  private static Random rand = new Random();
+
+  public final static String reservationQ = "dedicated";
+
+  public static ReservationId getNewReservationId() {
+    return ReservationId.newInstance(rand.nextLong(), rand.nextLong());
+  }
+
+  public CapacityScheduler mockCapacityScheduler(int numContainers)
+      throws IOException {
+    // stolen from TestCapacityScheduler
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    setupQueueConfiguration(conf);
+
+    CapacityScheduler cs = Mockito.spy(new CapacityScheduler());
+    cs.setConf(new YarnConfiguration());
+    RMContext mockRmContext =
+        Mockito.spy(new RMContextImpl(null, null, null, null, null, null,
+            new RMContainerTokenSecretManager(conf),
+            new NMTokenSecretManagerInRM(conf),
+            new ClientToAMTokenSecretManagerInRM(), null));
+    cs.setRMContext(mockRmContext);
+    try {
+      cs.serviceInit(conf);
+    } catch (Exception e) {
+      Assert.fail(e.getMessage());
+    }
+    when(mockRmContext.getScheduler()).thenReturn(cs);
+    Resource r = Resource.newInstance(numContainers * 1024, numContainers);
+    doReturn(r).when(cs).getClusterResource();
+    return cs;
+  }
+
+  public static void setupQueueConfiguration(CapacitySchedulerConfiguration conf) {
+    // Define default queue
+    final String defQ = CapacitySchedulerConfiguration.ROOT + ".default";
+    conf.setCapacity(defQ, 10);
+
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {
+        "default", "a", reservationQ });
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+
+    final String dedicated =
+        CapacitySchedulerConfiguration.ROOT
+            + CapacitySchedulerConfiguration.DOT + reservationQ;
+    conf.setCapacity(dedicated, 80);
+    // Set as reservation queue
+    conf.setReservableQueue(dedicated, true);
+
+    // Define 2nd-level queues
+    final String A1 = A + ".a1";
+    final String A2 = A + ".a2";
+    conf.setQueues(A, new String[] { "a1", "a2" });
+    conf.setCapacity(A1, 30);
+    conf.setCapacity(A2, 70);
+  }
+
+  public String getFullReservationQueueName() {
+    return CapacitySchedulerConfiguration.ROOT
+        + CapacitySchedulerConfiguration.DOT + reservationQ;
+  }
+
+  public String getreservationQueueName() {
+    return reservationQ;
+  }
+
+  public void updateQueueConfiguration(CapacitySchedulerConfiguration conf,
+      String newQ) {
+    // Define default queue
+    final String prefix =
+        CapacitySchedulerConfiguration.ROOT
+            + CapacitySchedulerConfiguration.DOT;
+    final String defQ = prefix + "default";
+    conf.setCapacity(defQ, 5);
+
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {
+        "default", "a", reservationQ, newQ });
+
+    final String A = prefix + "a";
+    conf.setCapacity(A, 5);
+
+    final String dedicated = prefix + reservationQ;
+    conf.setCapacity(dedicated, 80);
+    // Set as reservation queue
+    conf.setReservableQueue(dedicated, true);
+
+    conf.setCapacity(prefix + newQ, 10);
+    // Set as reservation queue
+    conf.setReservableQueue(prefix + newQ, true);
+
+    // Define 2nd-level queues
+    final String A1 = A + ".a1";
+    final String A2 = A + ".a2";
+    conf.setQueues(A, new String[] { "a1", "a2" });
+    conf.setCapacity(A1, 30);
+    conf.setCapacity(A2, 70);
+  }
+
+  public static ReservationDefinition generateRandomRR(Random rand, long i) {
+    rand.setSeed(i);
+    long now = System.currentTimeMillis();
+
+    // start time at random in the next 12 hours
+    long arrival = rand.nextInt(12 * 3600 * 1000);
+    // deadline at random in the next day
+    long deadline = arrival + rand.nextInt(24 * 3600 * 1000);
+
+    // create a request with a single atomic ask
+    ReservationDefinition rr = new ReservationDefinitionPBImpl();
+    rr.setArrival(now + arrival);
+    rr.setDeadline(now + deadline);
+
+    int gang = 1 + rand.nextInt(9);
+    int par = (rand.nextInt(1000) + 1) * gang;
+    long dur = rand.nextInt(2 * 3600 * 1000); // random duration within 2h
+    ReservationRequest r =
+        ReservationRequest.newInstance(Resource.newInstance(1024, 1), par,
+            gang, dur);
+    ReservationRequests reqs = new ReservationRequestsPBImpl();
+    reqs.setReservationResources(Collections.singletonList(r));
+    rand.nextInt(3);
+    ReservationRequestInterpreter[] type =
+        ReservationRequestInterpreter.values();
+    reqs.setInterpreter(type[rand.nextInt(type.length)]);
+    rr.setReservationRequests(reqs);
+
+    return rr;
+
+  }
+
+  public static ReservationDefinition generateBigRR(Random rand, long i) {
+    rand.setSeed(i);
+    long now = System.currentTimeMillis();
+
+    // start time at random in the next 2 hours
+    long arrival = rand.nextInt(2 * 3600 * 1000);
+    // deadline at random in the next day
+    long deadline = rand.nextInt(24 * 3600 * 1000);
+
+    // create a request with a single atomic ask
+    ReservationDefinition rr = new ReservationDefinitionPBImpl();
+    rr.setArrival(now + arrival);
+    rr.setDeadline(now + deadline);
+
+    int gang = 1;
+    int par = 100000; // 100k tasks
+    long dur = rand.nextInt(60 * 1000); // 1min tasks
+    ReservationRequest r =
+        ReservationRequest.newInstance(Resource.newInstance(1024, 1), par,
+            gang, dur);
+    ReservationRequests reqs = new ReservationRequestsPBImpl();
+    reqs.setReservationResources(Collections.singletonList(r));
+    rand.nextInt(3);
+    ReservationRequestInterpreter[] type =
+        ReservationRequestInterpreter.values();
+    reqs.setInterpreter(type[rand.nextInt(type.length)]);
+    rr.setReservationRequests(reqs);
+
+    return rr;
+  }
+
+  public static Map<ReservationInterval, ReservationRequest> generateAllocation(
+      long startTime, long step, int[] alloc) {
+    Map<ReservationInterval, ReservationRequest> req =
+        new TreeMap<ReservationInterval, ReservationRequest>();
+    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]));
+    }
+    return req;
+  }
+
+}


[12/16] git commit: MAPREDUCE-6103.Adding reservation APIs to MR resource manager delegate. Contributed by Subru Krishnan and Carlo Curino. (cherry picked from commit aa92dd45f2d8c89a8a17ad2e4449aa3ff08bc53a)

Posted by cd...@apache.org.
MAPREDUCE-6103.Adding reservation APIs to MR resource manager delegate. Contributed by Subru Krishnan and Carlo Curino.
(cherry picked from commit aa92dd45f2d8c89a8a17ad2e4449aa3ff08bc53a)


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

Branch: refs/heads/trunk
Commit: 3f282762d1afc916de9207d3adeda852ca344853
Parents: 6261f7c
Author: subru <su...@outlook.com>
Authored: Wed Sep 24 18:01:38 2014 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Oct 3 15:42:59 2014 -0700

----------------------------------------------------------------------
 YARN-1051-CHANGES.txt                           |  3 +++
 .../java/org/apache/hadoop/mapreduce/Job.java   | 21 +++++++++++++++++
 .../apache/hadoop/mapreduce/JobSubmitter.java   |  8 +++++++
 .../apache/hadoop/mapreduce/MRJobConfig.java    |  2 ++
 .../hadoop/mapred/ResourceMgrDelegate.java      | 24 ++++++++++++++++++++
 .../org/apache/hadoop/mapred/YARNRunner.java    | 22 ++++++++++++++++++
 .../hadoop/mapred/TestClientRedirect.java       | 24 ++++++++++++++++++++
 7 files changed, 104 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f282762/YARN-1051-CHANGES.txt
----------------------------------------------------------------------
diff --git a/YARN-1051-CHANGES.txt b/YARN-1051-CHANGES.txt
index c4106b2..6a27197 100644
--- a/YARN-1051-CHANGES.txt
+++ b/YARN-1051-CHANGES.txt
@@ -23,3 +23,6 @@ subsystem with the scheduler. (Subru Krishnan and Carlo Curino  via subru)
 
 YARN-2080. Integrating reservation system with ResourceManager and 
 client-RM protocol. (Subru Krishnan and Carlo Curino  via subru)
+
+MAPREDUCE-6103. Adding reservation APIs to MR resource manager
+delegate. (Subru Krishnan and Carlo Curino  via subru)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f282762/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
index 3f8d139..cfc3437 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
 import org.apache.hadoop.mapreduce.task.JobContextImpl;
 import org.apache.hadoop.mapreduce.util.ConfigUtil;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 
 /**
  * The job submitter's view of the Job.
@@ -112,6 +113,7 @@ public class Job extends JobContextImpl implements JobContext {
   private JobStatus status;
   private long statustime;
   private Cluster cluster;
+  private ReservationId reservationId;
 
   /**
    * @deprecated Use {@link #getInstance()}
@@ -1523,5 +1525,24 @@ public class Job extends JobContextImpl implements JobContext {
     updateStatus();
     return status.isUber();
   }
+
+  /**
+   * Get the reservation to which the job is submitted to, if any
+   *
+   * @return the reservationId the identifier of the job's reservation, null if
+   *         the job does not have any reservation associated with it
+   */
+  public ReservationId getReservationId() {
+    return reservationId;
+  }
+
+  /**
+   * Set the reservation to which the job is submitted to
+   *
+   * @param reservationId the reservationId to set
+   */
+  public void setReservationId(ReservationId reservationId) {
+    this.reservationId = reservationId;
+  }
   
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f282762/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
index 6cd569a..d80521c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.QueueACL;
+
 import static org.apache.hadoop.mapred.QueueManager.toFullPropertyName;
 
 import org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager;
@@ -60,6 +61,7 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.codehaus.jackson.JsonParseException;
 import org.codehaus.jackson.map.JsonMappingException;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -427,6 +429,12 @@ class JobSubmitter {
             trackingIds.toArray(new String[trackingIds.size()]));
       }
 
+      // Set reservation info if it exists
+      ReservationId reservationId = job.getReservationId();
+      if (reservationId != null) {
+        conf.set(MRJobConfig.RESERVATION_ID, reservationId.toString());
+      }
+
       // Write job file to submit dir
       writeConf(conf, submitJobFile);
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f282762/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 562120f..5b623b5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -63,6 +63,8 @@ public interface MRJobConfig {
 
   public static final String QUEUE_NAME = "mapreduce.job.queuename";
 
+  public static final String RESERVATION_ID = "mapreduce.job.reservation.id";
+
   public static final String JOB_TAGS = "mapreduce.job.tags";
 
   public static final String JVM_NUMTASKS_TORUN = "mapreduce.job.jvm.numtasks";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f282762/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
index b76d0f3..803390f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
@@ -43,6 +43,12 @@ import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -406,4 +412,22 @@ public class ResourceMgrDelegate extends YarnClient {
       throws YarnException, IOException {
     client.moveApplicationAcrossQueues(appId, queue);
   }
+
+  @Override
+  public ReservationSubmissionResponse submitReservation(
+      ReservationSubmissionRequest request) throws YarnException, IOException {
+    return client.submitReservation(request);
+  }
+
+  @Override
+  public ReservationUpdateResponse updateReservation(
+      ReservationUpdateRequest request) throws YarnException, IOException {
+    return client.updateReservation(request);
+  }
+
+  @Override
+  public ReservationDeleteResponse deleteReservation(
+      ReservationDeleteRequest request) throws YarnException, IOException {
+    return client.deleteReservation(request);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f282762/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
index 5120c85..9419d03 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
@@ -76,6 +76,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
@@ -489,6 +490,26 @@ public class YARNRunner implements ClientProtocol {
     appContext.setQueue(                                       // Queue name
         jobConf.get(JobContext.QUEUE_NAME,
         YarnConfiguration.DEFAULT_QUEUE_NAME));
+    // add reservationID if present
+    ReservationId reservationID = null;
+    try {
+      reservationID =
+          ReservationId.parseReservationId(jobConf
+              .get(JobContext.RESERVATION_ID));
+    } catch (NumberFormatException e) {
+      // throw exception as reservationid as is invalid
+      String errMsg =
+          "Invalid reservationId: " + jobConf.get(JobContext.RESERVATION_ID)
+              + " specified for the app: " + applicationId;
+      LOG.warn(errMsg);
+      throw new IOException(errMsg);
+    }
+    if (reservationID != null) {
+      appContext.setReservationID(reservationID);
+      LOG.info("SUBMITTING ApplicationSubmissionContext app:" + applicationId
+          + " to queue:" + appContext.getQueue() + " with reservationId:"
+          + appContext.getReservationID());
+    }
     appContext.setApplicationName(                             // Job name
         jobConf.get(JobContext.JOB_NAME,
         YarnConfiguration.DEFAULT_APPLICATION_NAME));
@@ -503,6 +524,7 @@ public class YARNRunner implements ClientProtocol {
     if (tagsFromConf != null && !tagsFromConf.isEmpty()) {
       appContext.setApplicationTags(new HashSet<String>(tagsFromConf));
     }
+
     return appContext;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f282762/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
index 69ede3a..5663a81 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
@@ -102,6 +102,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesReq
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -397,6 +403,24 @@ public class TestClientRedirect {
         throws YarnException, IOException {
       return null;
     }
+
+    @Override
+    public ReservationSubmissionResponse submitReservation(
+        ReservationSubmissionRequest request) throws YarnException, IOException {
+      return null;
+    }
+
+    @Override
+    public ReservationUpdateResponse updateReservation(
+        ReservationUpdateRequest request) throws YarnException, IOException {
+      return null;
+    }
+
+    @Override
+    public ReservationDeleteResponse deleteReservation(
+        ReservationDeleteRequest request) throws YarnException, IOException {
+      return null;
+    }
   }
 
   class HistoryService extends AMService implements HSClientProtocol {


[15/16] git commit: YARN-2611. Fixing jenkins findbugs warning and TestRMWebServicesCapacitySched for branch YARN-1051. Contributed by Subru Krishnan and Carlo Curino. (cherry picked from commit c47464aba407d1dafe10be23fe454f0489cc4367)

Posted by cd...@apache.org.
YARN-2611. Fixing jenkins findbugs warning and TestRMWebServicesCapacitySched for branch YARN-1051. Contributed by Subru Krishnan and Carlo Curino.
(cherry picked from commit c47464aba407d1dafe10be23fe454f0489cc4367)


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

Branch: refs/heads/trunk
Commit: a2986234be4e02f9ccb589f9ff5f7ffb28bc6400
Parents: 5e10a13
Author: subru <su...@outlook.com>
Authored: Fri Sep 26 10:48:12 2014 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Oct 3 15:43:23 2014 -0700

----------------------------------------------------------------------
 YARN-1051-CHANGES.txt                           |  3 ++
 .../reservation/CapacityOverTimePolicy.java     |  7 ++++-
 .../CapacitySchedulerPlanFollower.java          |  2 +-
 .../reservation/InMemoryPlan.java               |  6 ++--
 .../reservation/ReservationInterval.java        | 30 ++++++++++++++++++++
 .../reservation/SimpleCapacityReplanner.java    |  5 ++++
 .../webapp/TestRMWebServicesCapacitySched.java  |  4 +--
 7 files changed, 50 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2986234/YARN-1051-CHANGES.txt
----------------------------------------------------------------------
diff --git a/YARN-1051-CHANGES.txt b/YARN-1051-CHANGES.txt
index 7620cc3..5cd1136 100644
--- a/YARN-1051-CHANGES.txt
+++ b/YARN-1051-CHANGES.txt
@@ -29,3 +29,6 @@ delegate. (Subru Krishnan and Carlo Curino  via subru)
 
 YARN-2576. Fixing compilation, javadocs and audit issues to pass
 test patch in branch. (Subru Krishnan and Carlo Curino  via subru)
+
+YARN-2611. Fixing jenkins findbugs warning and TestRMWebServicesCapacitySched
+for branch YARN-1051. (Subru Krishnan and Carlo Curino  via subru)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2986234/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.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/CapacityOverTimePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.java
index 680f9ae..7552e8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacityOverTimePolicy.java
@@ -62,6 +62,11 @@ public class CapacityOverTimePolicy implements SharingPolicy {
   // it should be easy to remove this limitation
   @Override
   public void init(String reservationQueuePath, Configuration conf) {
+    if (!(conf instanceof CapacitySchedulerConfiguration)) {
+      throw new IllegalArgumentException("Unexpected conf type: "
+          + conf.getClass().getSimpleName() + " only supported conf is: "
+          + CapacitySchedulerConfiguration.class.getSimpleName());
+    }
     this.conf = (CapacitySchedulerConfiguration) conf;
     validWindow = this.conf.getReservationWindow(reservationQueuePath);
     maxInst = this.conf.getInstantaneousMaxCapacity(reservationQueuePath) / 100;
@@ -203,7 +208,7 @@ public class CapacityOverTimePolicy implements SharingPolicy {
    * The comparison/multiplication behaviors of IntegralResource are consistent
    * with the DefaultResourceCalculator.
    */
-  public class IntegralResource {
+  private static class IntegralResource {
     long memory;
     long vcores;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2986234/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.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/CapacitySchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java
index cfa172c..0c0fbc0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java
@@ -326,7 +326,7 @@ public class CapacitySchedulerPlanFollower implements PlanFollower {
     return currentReservations;
   }
 
-  private class ReservationAllocationComparator implements
+  private static class ReservationAllocationComparator implements
       Comparator<ReservationAllocation> {
     CapacityScheduler scheduler;
     long now;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2986234/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 8394a49..ce2e7d7 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
@@ -138,7 +138,7 @@ class InMemoryPlan implements Plan {
       rleSparseVector.removeInterval(r.getKey(), r.getValue());
     }
     if (resAlloc.isEmpty()) {
-      userResourceAlloc.remove(resAlloc);
+      userResourceAlloc.remove(user);
     }
   }
 
@@ -311,9 +311,9 @@ class InMemoryPlan implements Plan {
   public void archiveCompletedReservations(long tick) {
     // Since we are looking for old reservations, read lock is optimal
     LOG.debug("Running archival at time: {}", tick);
-    readLock.lock();
     List<InMemoryReservationAllocation> expiredReservations =
         new ArrayList<InMemoryReservationAllocation>();
+    readLock.lock();
     // archive reservations and delete the ones which are beyond
     // the reservation policy "window"
     try {
@@ -351,9 +351,9 @@ class InMemoryPlan implements Plan {
 
   @Override
   public Set<ReservationAllocation> getReservationsAtTime(long tick) {
-    readLock.lock();
     ReservationInterval searchInterval =
         new ReservationInterval(tick, Long.MAX_VALUE);
+    readLock.lock();
     try {
       SortedMap<ReservationInterval, Set<InMemoryReservationAllocation>> reservations =
           currentReservations.headMap(searchInterval, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2986234/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInterval.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/ReservationInterval.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInterval.java
index aabbcb3..1ca2969 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInterval.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInterval.java
@@ -78,6 +78,36 @@ public class ReservationInterval implements Comparable<ReservationInterval> {
     }
   }
 
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + (int) (endTime ^ (endTime >>> 32));
+    result = prime * result + (int) (startTime ^ (startTime >>> 32));
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof ReservationInterval)) {
+      return false;
+    }
+    ReservationInterval other = (ReservationInterval) obj;
+    if (endTime != other.endTime) {
+      return false;
+    }
+    if (startTime != other.startTime) {
+      return false;
+    }
+    return true;
+  }
+
   public String toString() {
     return "[" + startTime + ", " + endTime + "]";
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2986234/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SimpleCapacityReplanner.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/SimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SimpleCapacityReplanner.java
index 5378405..e38dd3cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SimpleCapacityReplanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/SimpleCapacityReplanner.java
@@ -70,6 +70,11 @@ public class SimpleCapacityReplanner implements Planner {
 
   @Override
   public void init(String planQueueName, Configuration conf) {
+    if (!(conf instanceof CapacitySchedulerConfiguration)) {
+      throw new IllegalArgumentException("Unexpected conf type: "
+          + conf.getClass().getSimpleName() + " only supported conf is: "
+          + CapacitySchedulerConfiguration.class.getSimpleName());
+    }
     this.lengthOfCheckZone =
         ((CapacitySchedulerConfiguration) conf)
             .getEnforcementWindow(planQueueName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2986234/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.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/webapp/TestRMWebServicesCapacitySched.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
index e57e5cf..e58c30f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
@@ -357,10 +357,10 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
   private void verifySubQueue(JSONObject info, String q, 
       float parentAbsCapacity, float parentAbsMaxCapacity)
       throws JSONException, Exception {
-    int numExpectedElements = 11;
+    int numExpectedElements = 12;
     boolean isParentQueue = true;
     if (!info.has("queues")) {
-      numExpectedElements = 21;
+      numExpectedElements = 22;
       isParentQueue = false;
     }
     assertEquals("incorrect number of elements", numExpectedElements, info.length());