You are viewing a plain text version of this content. The canonical link for it is here.
Posted to yarn-commits@hadoop.apache.org by vi...@apache.org on 2014/05/22 07:32:27 UTC

svn commit: r1596753 [1/2] - in /hadoop/common/trunk/hadoop-yarn-project: ./ hadoop-yarn/dev-support/ hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ hadoop...

Author: vinodkv
Date: Thu May 22 05:32:26 2014
New Revision: 1596753

URL: http://svn.apache.org/r1596753
Log:
YARN-2017. Merged some of the common scheduler code. Contributed by Jian He.

Modified:
    hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
    hadoop/common/trunk/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
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.java
    hadoop/common/trunk/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
    hadoop/common/trunk/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
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
    hadoop/common/trunk/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
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java

Modified: hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt Thu May 22 05:32:26 2014
@@ -88,6 +88,8 @@ Release 2.5.0 - UNRELEASED
     YARN-1938. Added kerberos login for the Timeline Server. (Zhijie Shen via
     vinodkv)
 
+    YARN-2017. Merged some of the common scheduler code. (Jian He via vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES 

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml Thu May 22 05:32:26 2014
@@ -142,6 +142,17 @@
     <Class name="org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService" />
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
+  <!-- Inconsistent sync warning - minimumAllocation is only initialized once and never changed -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler" />
+    <Field name="minimumAllocation" />
+    <Bug pattern="IS2_INCONSISTENT_SYNC" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSSchedulerNode" />
+    <Method name="reserveResource" />
+    <Bug pattern="BC_UNCONFIRMED_CAST" /> 
+  </Match>
   <!-- Inconsistent sync warning - reinitialize read from other queue does not need sync-->
   <Match>
     <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue" />
@@ -178,12 +189,6 @@
     <Field name="scheduleAsynchronously" />
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
-  <!-- Inconsistent sync warning - minimumAllocation is only initialized once and never changed -->
-  <Match>
-    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler" />
-    <Field name="minimumAllocation" />
-    <Bug pattern="IS2_INCONSISTENT_SYNC" />
-  </Match>
   <!-- Inconsistent sync warning - numRetries is only initialized once and never changed -->
   <Match>
     <Class name="org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore" />

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java Thu May 22 05:32:26 2014
@@ -170,7 +170,7 @@ public class ProportionalCapacityPreempt
   public void editSchedule(){
     CSQueue root = scheduler.getRootQueue();
     Resource clusterResources =
-      Resources.clone(scheduler.getClusterResources());
+      Resources.clone(scheduler.getClusterResource());
     containerBasedPreemptOrKill(root, clusterResources);
   }
 

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java Thu May 22 05:32:26 2014
@@ -22,21 +22,41 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
+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.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+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.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-public abstract class AbstractYarnScheduler implements ResourceScheduler {
+public abstract class AbstractYarnScheduler
+    <T extends SchedulerApplicationAttempt, N extends SchedulerNode>
+    implements ResourceScheduler {
+
+  private static final Log LOG = LogFactory.getLog(AbstractYarnScheduler.class);
+
+  // Nodes in the cluster, indexed by NodeId
+  protected Map<NodeId, N> nodes =
+      new ConcurrentHashMap<NodeId, N>();
+
+  // Whole capacity of the cluster
+  protected Resource clusterResource = Resource.newInstance(0, 0);
+
+  protected Resource minimumAllocation;
+  protected Resource maximumAllocation;
 
   protected RMContext rmContext;
-  protected Map<ApplicationId, SchedulerApplication> applications;
+  protected Map<ApplicationId, SchedulerApplication<T>> applications;
   protected final static List<Container> EMPTY_CONTAINER_LIST =
       new ArrayList<Container>();
   protected static final Allocation EMPTY_ALLOCATION = new Allocation(
@@ -45,7 +65,7 @@ public abstract class AbstractYarnSchedu
   public synchronized List<Container> getTransferredContainers(
       ApplicationAttemptId currentAttempt) {
     ApplicationId appId = currentAttempt.getApplicationId();
-    SchedulerApplication app = applications.get(appId);
+    SchedulerApplication<T> app = applications.get(appId);
     List<Container> containerList = new ArrayList<Container>();
     RMApp appImpl = this.rmContext.getRMApps().get(appId);
     if (appImpl.getApplicationSubmissionContext().getUnmanagedAM()) {
@@ -64,10 +84,75 @@ public abstract class AbstractYarnSchedu
     return containerList;
   }
 
-  public Map<ApplicationId, SchedulerApplication> getSchedulerApplications() {
+  public Map<ApplicationId, SchedulerApplication<T>>
+      getSchedulerApplications() {
     return applications;
   }
-  
+
+  @Override
+  public Resource getClusterResource() {
+    return clusterResource;
+  }
+
+  @Override
+  public Resource getMinimumResourceCapability() {
+    return minimumAllocation;
+  }
+
+  @Override
+  public Resource getMaximumResourceCapability() {
+    return maximumAllocation;
+  }
+
+  public T getApplicationAttempt(ApplicationAttemptId applicationAttemptId) {
+    SchedulerApplication<T> app =
+        applications.get(applicationAttemptId.getApplicationId());
+    return app == null ? null : app.getCurrentAppAttempt();
+  }
+
+  @Override
+  public SchedulerAppReport getSchedulerAppInfo(
+      ApplicationAttemptId appAttemptId) {
+    SchedulerApplicationAttempt attempt = getApplicationAttempt(appAttemptId);
+    if (attempt == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Request for appInfo of unknown attempt " + appAttemptId);
+      }
+      return null;
+    }
+    return new SchedulerAppReport(attempt);
+  }
+
+  @Override
+  public ApplicationResourceUsageReport getAppResourceUsageReport(
+      ApplicationAttemptId appAttemptId) {
+    SchedulerApplicationAttempt attempt = getApplicationAttempt(appAttemptId);
+    if (attempt == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Request for appInfo of unknown attempt " + appAttemptId);
+      }
+      return null;
+    }
+    return attempt.getResourceUsageReport();
+  }
+
+  public T getCurrentAttemptForContainer(ContainerId containerId) {
+    return getApplicationAttempt(containerId.getApplicationAttemptId());
+  }
+
+  @Override
+  public RMContainer getRMContainer(ContainerId containerId) {
+    SchedulerApplicationAttempt attempt =
+        getCurrentAttemptForContainer(containerId);
+    return (attempt == null) ? null : attempt.getRMContainer(containerId);
+  }
+
+  @Override
+  public SchedulerNodeReport getNodeReport(NodeId nodeId) {
+    N node = nodes.get(nodeId);
+    return node == null ? null : new SchedulerNodeReport(node);
+  }
+
   @Override
   public String moveApplication(ApplicationId appId, String newQueue)
       throws YarnException {

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java Thu May 22 05:32:26 2014
@@ -23,11 +23,11 @@ import org.apache.hadoop.yarn.server.res
 
 @Private
 @Unstable
-public class SchedulerApplication {
+public class SchedulerApplication<T extends SchedulerApplicationAttempt> {
 
   private Queue queue;
   private final String user;
-  private SchedulerApplicationAttempt currentAttempt;
+  private T currentAttempt;
 
   public SchedulerApplication(Queue queue, String user) {
     this.queue = queue;
@@ -46,11 +46,11 @@ public class SchedulerApplication {
     return user;
   }
 
-  public SchedulerApplicationAttempt getCurrentAppAttempt() {
+  public T getCurrentAppAttempt() {
     return currentAttempt;
   }
 
-  public void setCurrentAppAttempt(SchedulerApplicationAttempt currentAttempt) {
+  public void setCurrentAppAttempt(T currentAttempt) {
     this.currentAttempt = currentAttempt;
   }
 

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java Thu May 22 05:32:26 2014
@@ -18,11 +18,27 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import com.google.common.base.Preconditions;
 
 /**
  * Represents a YARN Cluster Node from the viewpoint of the scheduler.
@@ -31,59 +47,231 @@ import org.apache.hadoop.yarn.conf.YarnC
 @Unstable
 public abstract class SchedulerNode {
 
+  private static final Log LOG = LogFactory.getLog(SchedulerNode.class);
+
+  private Resource availableResource = Resource.newInstance(0, 0);
+  private Resource usedResource = Resource.newInstance(0, 0);
+  private Resource totalResourceCapability;
+  private RMContainer reservedContainer;
+  private volatile int numContainers;
+
+
+  /* set of containers that are allocated containers */
+  private final Map<ContainerId, RMContainer> launchedContainers =
+      new HashMap<ContainerId, RMContainer>();
+
+  private final RMNode rmNode;
+  private final String nodeName;
+
+  public SchedulerNode(RMNode node, boolean usePortForNodeName) {
+    this.rmNode = node;
+    this.availableResource = Resources.clone(node.getTotalCapability());
+    this.totalResourceCapability = Resources.clone(node.getTotalCapability());
+    if (usePortForNodeName) {
+      nodeName = rmNode.getHostName() + ":" + node.getNodeID().getPort();
+    } else {
+      nodeName = rmNode.getHostName();
+    }
+  }
+
+  public RMNode getRMNode() {
+    return this.rmNode;
+  }
+
+  /**
+   * Get the ID of the node which contains both its hostname and port.
+   * 
+   * @return the ID of the node
+   */
+  public NodeId getNodeID() {
+    return this.rmNode.getNodeID();
+  }
+
+  public String getHttpAddress() {
+    return this.rmNode.getHttpAddress();
+  }
+
   /**
    * Get the name of the node for scheduling matching decisions.
    * <p/>
-   * Typically this is the 'hostname' reported by the node, but it could be 
-   * configured to be 'hostname:port' reported by the node via the 
+   * Typically this is the 'hostname' reported by the node, but it could be
+   * configured to be 'hostname:port' reported by the node via the
    * {@link YarnConfiguration#RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME} constant.
    * The main usecase of this is Yarn minicluster to be able to differentiate
    * node manager instances by their port number.
    * 
    * @return name of the node for scheduling matching decisions.
    */
-  public abstract String getNodeName();
-  
+  public String getNodeName() {
+    return nodeName;
+  }
+
   /**
    * Get rackname.
+   * 
    * @return rackname
    */
-  public abstract String getRackName();
-  
+  public String getRackName() {
+    return this.rmNode.getRackName();
+  }
+
   /**
-   * Get used resources on the node.
-   * @return used resources on the node
+   * The Scheduler has allocated containers on this node to the given
+   * application.
+   * 
+   * @param applicationId
+   *          application
+   * @param rmContainer
+   *          allocated container
    */
-  public abstract Resource getUsedResource();
+  public synchronized void allocateContainer(ApplicationId applicationId,
+      RMContainer rmContainer) {
+    Container container = rmContainer.getContainer();
+    deductAvailableResource(container.getResource());
+    ++numContainers;
+
+    launchedContainers.put(container.getId(), rmContainer);
+
+    LOG.info("Assigned container " + container.getId() + " of capacity "
+        + container.getResource() + " on host " + rmNode.getNodeAddress()
+        + ", which currently has " + numContainers + " containers, "
+        + getUsedResource() + " used and " + getAvailableResource()
+        + " available");
+  }
 
   /**
    * Get available resources on the node.
+   * 
    * @return available resources on the node
    */
-  public abstract Resource getAvailableResource();
+  public synchronized Resource getAvailableResource() {
+    return this.availableResource;
+  }
 
   /**
-   * Get number of active containers on the node.
-   * @return number of active containers on the node
-   */
-  public abstract int getNumContainers();
-  
-  /**
-   * Apply delta resource on node's available resource.
-   * @param deltaResource the delta of resource need to apply to node
+   * Get used resources on the node.
+   * 
+   * @return used resources on the node
    */
-  public abstract void applyDeltaOnAvailableResource(Resource deltaResource);
+  public synchronized Resource getUsedResource() {
+    return this.usedResource;
+  }
 
   /**
    * Get total resources on the node.
+   * 
    * @return total resources on the node.
    */
-  public abstract Resource getTotalResource();
-  
+  public Resource getTotalResource() {
+    return this.totalResourceCapability;
+  }
+
+  private synchronized boolean isValidContainer(Container c) {
+    if (launchedContainers.containsKey(c.getId())) {
+      return true;
+    }
+    return false;
+  }
+
+  private synchronized void updateResource(Container container) {
+    addAvailableResource(container.getResource());
+    --numContainers;
+  }
+
   /**
-   * Get the ID of the node which contains both its hostname and port.
-   * @return the ID of the node
+   * Release an allocated container on this node.
+   * 
+   * @param container
+   *          container to be released
+   */
+  public synchronized void releaseContainer(Container container) {
+    if (!isValidContainer(container)) {
+      LOG.error("Invalid container released " + container);
+      return;
+    }
+
+    /* remove the containers from the nodemanger */
+    if (null != launchedContainers.remove(container.getId())) {
+      updateResource(container);
+    }
+
+    LOG.info("Released container " + container.getId() + " of capacity "
+        + container.getResource() + " on host " + rmNode.getNodeAddress()
+        + ", which currently has " + numContainers + " containers, "
+        + getUsedResource() + " used and " + getAvailableResource()
+        + " available" + ", release resources=" + true);
+  }
+
+  private synchronized void addAvailableResource(Resource resource) {
+    if (resource == null) {
+      LOG.error("Invalid resource addition of null resource for "
+          + rmNode.getNodeAddress());
+      return;
+    }
+    Resources.addTo(availableResource, resource);
+    Resources.subtractFrom(usedResource, resource);
+  }
+
+  private synchronized void deductAvailableResource(Resource resource) {
+    if (resource == null) {
+      LOG.error("Invalid deduction of null resource for "
+          + rmNode.getNodeAddress());
+      return;
+    }
+    Resources.subtractFrom(availableResource, resource);
+    Resources.addTo(usedResource, resource);
+  }
+
+  /**
+   * Reserve container for the attempt on this node.
+   */
+  public abstract void reserveResource(SchedulerApplicationAttempt attempt,
+      Priority priority, RMContainer container);
+
+  /**
+   * Unreserve resources on this node.
+   */
+  public abstract void unreserveResource(SchedulerApplicationAttempt attempt);
+
+  @Override
+  public String toString() {
+    return "host: " + rmNode.getNodeAddress() + " #containers="
+        + getNumContainers() + " available="
+        + getAvailableResource().getMemory() + " used="
+        + getUsedResource().getMemory();
+  }
+
+  /**
+   * Get number of active containers on the node.
+   * 
+   * @return number of active containers on the node
    */
-  public abstract NodeId getNodeID();
+  public int getNumContainers() {
+    return numContainers;
+  }
+
+  public synchronized List<RMContainer> getRunningContainers() {
+    return new ArrayList<RMContainer>(launchedContainers.values());
+  }
+
+  public synchronized RMContainer getReservedContainer() {
+    return reservedContainer;
+  }
+
+  protected synchronized void
+      setReservedContainer(RMContainer reservedContainer) {
+    this.reservedContainer = reservedContainer;
+  }
 
+  /**
+   * Apply delta resource on node's available resource.
+   * 
+   * @param deltaResource
+   *          the delta of resource need to apply to node
+   */
+  public synchronized void
+      applyDeltaOnAvailableResource(Resource deltaResource) {
+    // we can only adjust available resource if total resource is changed.
+    Resources.addTo(this.availableResource, deltaResource);
+  }
 }

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java Thu May 22 05:32:26 2014
@@ -69,7 +69,15 @@ public interface YarnScheduler extends E
   @Public
   @Stable
   public List<QueueUserACLInfo> getQueueUserAclInfo();
-  
+
+  /**
+   * Get the whole resource capacity of the cluster.
+   * @return the whole resource capacity of the cluster.
+   */
+  @LimitedPrivate("yarn")
+  @Unstable
+  public Resource getClusterResource();
+
   /**
    * Get minimum allocatable {@link Resource}.
    * @return minimum allocatable resource
@@ -182,7 +190,7 @@ public interface YarnScheduler extends E
   @LimitedPrivate("yarn")
   @Unstable
   public RMContainer getRMContainer(ContainerId containerId);
-  
+
   /**
    * Moves the given application to the given queue
    * @param appId

Modified: hadoop/common/trunk/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
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/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?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/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 (original)
+++ hadoop/common/trunk/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 Thu May 22 05:32:26 2014
@@ -41,7 +41,6 @@ import org.apache.hadoop.security.Access
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -49,11 +48,9 @@ import org.apache.hadoop.yarn.api.record
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
-import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -75,9 +72,7 @@ import org.apache.hadoop.yarn.server.res
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
@@ -100,9 +95,9 @@ import com.google.common.annotations.Vis
 @LimitedPrivate("yarn")
 @Evolving
 @SuppressWarnings("unchecked")
-public class CapacityScheduler extends AbstractYarnScheduler
-  implements PreemptableResourceScheduler, CapacitySchedulerContext,
-             Configurable {
+public class CapacityScheduler extends
+    AbstractYarnScheduler<FiCaSchedulerApp, FiCaSchedulerNode> implements
+    PreemptableResourceScheduler, CapacitySchedulerContext, Configurable {
 
   private static final Log LOG = LogFactory.getLog(CapacityScheduler.class);
 
@@ -182,16 +177,8 @@ public class CapacityScheduler extends A
 
   private Map<String, CSQueue> queues = new ConcurrentHashMap<String, CSQueue>();
 
-  private Map<NodeId, FiCaSchedulerNode> nodes = 
-      new ConcurrentHashMap<NodeId, FiCaSchedulerNode>();
-
-  private Resource clusterResource = 
-    RecordFactoryProvider.getRecordFactory(null).newRecordInstance(Resource.class);
   private int numNodeManagers = 0;
 
-  private Resource minimumAllocation;
-  private Resource maximumAllocation;
-
   private boolean initialized = false;
 
   private ResourceCalculator calculator;
@@ -231,16 +218,6 @@ public class CapacityScheduler extends A
   }
 
   @Override
-  public Resource getMinimumResourceCapability() {
-    return minimumAllocation;
-  }
-
-  @Override
-  public Resource getMaximumResourceCapability() {
-    return maximumAllocation;
-  }
-
-  @Override
   public Comparator<FiCaSchedulerApp> getApplicationComparator() {
     return applicationComparator;
   }
@@ -264,11 +241,6 @@ public class CapacityScheduler extends A
   public RMContext getRMContext() {
     return this.rmContext;
   }
-
-  @Override
-  public Resource getClusterResources() {
-    return clusterResource;
-  }
   
   @Override
   public synchronized void
@@ -283,7 +255,7 @@ public class CapacityScheduler extends A
       this.calculator = this.conf.getResourceCalculator();
       this.usePortForNodeName = this.conf.getUsePortForNodeName();
       this.applications =
-          new ConcurrentHashMap<ApplicationId, SchedulerApplication>();
+          new ConcurrentHashMap<ApplicationId, SchedulerApplication<FiCaSchedulerApp>>();
 
       initializeQueues(this.conf);
       
@@ -536,8 +508,8 @@ public class CapacityScheduler extends A
           .handle(new RMAppRejectedEvent(applicationId, ace.toString()));
       return;
     }
-    SchedulerApplication application =
-        new SchedulerApplication(queue, user);
+    SchedulerApplication<FiCaSchedulerApp> application =
+        new SchedulerApplication<FiCaSchedulerApp>(queue, user);
     applications.put(applicationId, application);
     LOG.info("Accepted application " + applicationId + " from user: " + user
         + ", in queue: " + queueName);
@@ -548,7 +520,7 @@ public class CapacityScheduler extends A
   private synchronized void addApplicationAttempt(
       ApplicationAttemptId applicationAttemptId,
       boolean transferStateFromPreviousAttempt) {
-    SchedulerApplication application =
+    SchedulerApplication<FiCaSchedulerApp> application =
         applications.get(applicationAttemptId.getApplicationId());
     CSQueue queue = (CSQueue) application.getQueue();
 
@@ -572,7 +544,8 @@ public class CapacityScheduler extends A
 
   private synchronized void doneApplication(ApplicationId applicationId,
       RMAppState finalState) {
-    SchedulerApplication application = applications.get(applicationId);
+    SchedulerApplication<FiCaSchedulerApp> application =
+        applications.get(applicationId);
     if (application == null){
       // The AppRemovedSchedulerEvent maybe sent on recovery for completed apps,
       // ignore it.
@@ -597,7 +570,7 @@ public class CapacityScheduler extends A
     		" finalState=" + rmAppAttemptFinalState);
     
     FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId);
-    SchedulerApplication application =
+    SchedulerApplication<FiCaSchedulerApp> application =
         applications.get(applicationAttemptId.getApplicationId());
 
     if (application == null || attempt == null) {
@@ -659,7 +632,7 @@ public class CapacityScheduler extends A
     
     // Sanity check
     SchedulerUtils.normalizeRequests(
-        ask, getResourceCalculator(), getClusterResources(),
+        ask, getResourceCalculator(), getClusterResource(),
         getMinimumResourceCapability(), maximumAllocation);
 
     // Release containers
@@ -822,7 +795,7 @@ public class CapacityScheduler extends A
 
     // Try to schedule more if there are no reservations to fulfill
     if (node.getReservedContainer() == null) {
-      if (Resources.greaterThanOrEqual(calculator, getClusterResources(),
+      if (Resources.greaterThanOrEqual(calculator, getClusterResource(),
           node.getAvailableResource(), minimumAllocation)) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Trying to schedule on node: " + node.getNodeName() +
@@ -942,7 +915,7 @@ public class CapacityScheduler extends A
   }
 
   private synchronized void removeNode(RMNode nodeInfo) {
-    FiCaSchedulerNode node = this.nodes.get(nodeInfo.getNodeID());
+    FiCaSchedulerNode node = nodes.get(nodeInfo.getNodeID());
     if (node == null) {
       return;
     }
@@ -1015,28 +988,10 @@ public class CapacityScheduler extends A
 
   @Lock(Lock.NoLock.class)
   @VisibleForTesting
-  public FiCaSchedulerApp getApplicationAttempt(
-      ApplicationAttemptId applicationAttemptId) {
-    SchedulerApplication app =
-        applications.get(applicationAttemptId.getApplicationId());
-    if (app != null) {
-      return (FiCaSchedulerApp) app.getCurrentAppAttempt();
-    }
-    return null;
-  }
-
-  @Override
-  public SchedulerAppReport getSchedulerAppInfo(
-      ApplicationAttemptId applicationAttemptId) {
-    FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
-    return app == null ? null : new SchedulerAppReport(app);
-  }
-  
   @Override
-  public ApplicationResourceUsageReport getAppResourceUsageReport(
+  public FiCaSchedulerApp getApplicationAttempt(
       ApplicationAttemptId applicationAttemptId) {
-    FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
-    return app == null ? null : app.getResourceUsageReport();
+    return super.getApplicationAttempt(applicationAttemptId);
   }
   
   @Lock(Lock.NoLock.class)
@@ -1048,24 +1003,6 @@ public class CapacityScheduler extends A
   Map<NodeId, FiCaSchedulerNode> getAllNodes() {
     return nodes;
   }
-  
-  @Override
-  public RMContainer getRMContainer(ContainerId containerId) {
-    FiCaSchedulerApp attempt = getCurrentAttemptForContainer(containerId);
-    return (attempt == null) ? null : attempt.getRMContainer(containerId);
-  }
-
-  @VisibleForTesting
-  public FiCaSchedulerApp getCurrentAttemptForContainer(
-      ContainerId containerId) {
-    SchedulerApplication app =
-        applications.get(containerId.getApplicationAttemptId()
-          .getApplicationId());
-    if (app != null) {
-      return (FiCaSchedulerApp) app.getCurrentAppAttempt();
-    }
-    return null;
-  }
 
   @Override
   @Lock(Lock.NoLock.class)
@@ -1074,12 +1011,6 @@ public class CapacityScheduler extends A
   }
 
   @Override
-  public SchedulerNodeReport getNodeReport(NodeId nodeId) {
-    FiCaSchedulerNode node = getNode(nodeId);
-    return node == null ? null : new SchedulerNodeReport(node);
-  }
-
-  @Override
   public void dropContainerReservation(RMContainer container) {
     if(LOG.isDebugEnabled()){
       LOG.debug("DROP_RESERVATION:" + container.toString());

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.java?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.java Thu May 22 05:32:26 2014
@@ -43,7 +43,7 @@ public interface CapacitySchedulerContex
 
   RMContext getRMContext();
   
-  Resource getClusterResources();
+  Resource getClusterResource();
 
   /**
    * Get the yarn configuration.

Modified: hadoop/common/trunk/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
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/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?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/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 (original)
+++ hadoop/common/trunk/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 Thu May 22 05:32:26 2014
@@ -174,12 +174,12 @@ public class LeafQueue implements CSQueu
     int maxActiveApplications = 
         CSQueueUtils.computeMaxActiveApplications(
             resourceCalculator,
-            cs.getClusterResources(), this.minimumAllocation,
+            cs.getClusterResource(), this.minimumAllocation,
             maxAMResourcePerQueuePercent, absoluteMaxCapacity);
     this.maxActiveAppsUsingAbsCap = 
             CSQueueUtils.computeMaxActiveApplications(
                 resourceCalculator,
-                cs.getClusterResources(), this.minimumAllocation,
+                cs.getClusterResource(), this.minimumAllocation,
                 maxAMResourcePerQueuePercent, absoluteCapacity);
     int maxActiveApplicationsPerUser = 
         CSQueueUtils.computeMaxActiveApplicationsPerUser(maxActiveAppsUsingAbsCap, userLimit, 
@@ -195,7 +195,7 @@ public class LeafQueue implements CSQueu
       cs.getConfiguration().getAcls(getQueuePath());
 
     setupQueueConfigs(
-        cs.getClusterResources(),
+        cs.getClusterResource(),
         capacity, absoluteCapacity, 
         maximumCapacity, absoluteMaxCapacity, 
         userLimit, userLimitFactor, 

Modified: hadoop/common/trunk/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
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/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?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/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 (original)
+++ hadoop/common/trunk/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 Thu May 22 05:32:26 2014
@@ -143,7 +143,7 @@ public class ParentQueue implements CSQu
     this.queueInfo.setQueueName(queueName);
     this.queueInfo.setChildQueues(new ArrayList<QueueInfo>());
 
-    setupQueueConfigs(cs.getClusterResources(),
+    setupQueueConfigs(cs.getClusterResource(),
         capacity, absoluteCapacity, 
         maximumCapacity, absoluteMaxCapacity, state, acls);
     

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java Thu May 22 05:32:26 2014
@@ -18,248 +18,84 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
 
 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.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
-import org.apache.hadoop.yarn.util.resource.Resources;
 
 public class FiCaSchedulerNode extends SchedulerNode {
 
   private static final Log LOG = LogFactory.getLog(FiCaSchedulerNode.class);
 
-  private static final RecordFactory recordFactory = RecordFactoryProvider
-      .getRecordFactory(null);
-
-  private Resource availableResource = recordFactory.newRecordInstance(Resource.class);
-  private Resource usedResource = recordFactory.newRecordInstance(Resource.class);
-  private Resource totalResourceCapability;
-
-  private volatile int numContainers;
-
-  private RMContainer reservedContainer;
-  
-  /* set of containers that are allocated containers */
-  private final Map<ContainerId, RMContainer> launchedContainers = 
-    new HashMap<ContainerId, RMContainer>();
-  
-  private final RMNode rmNode;
-  private final String nodeName;
-
   public FiCaSchedulerNode(RMNode node, boolean usePortForNodeName) {
-    this.rmNode = node;
-    this.availableResource.setMemory(node.getTotalCapability().getMemory());
-    this.availableResource.setVirtualCores(node.getTotalCapability().getVirtualCores());
-    totalResourceCapability =
-        Resource.newInstance(node.getTotalCapability().getMemory(), node
-            .getTotalCapability().getVirtualCores());
-    if (usePortForNodeName) {
-      nodeName = rmNode.getHostName() + ":" + node.getNodeID().getPort();
-    } else {
-      nodeName = rmNode.getHostName();
-    }
-  }
-
-  public RMNode getRMNode() {
-    return this.rmNode;
-  }
-
-  public NodeId getNodeID() {
-    return this.rmNode.getNodeID();
-  }
-
-  public String getHttpAddress() {
-    return this.rmNode.getHttpAddress();
-  }
-
-  @Override
-  public String getNodeName() {
-    return nodeName;
-  }
-
-  @Override
-  public String getRackName() {
-    return this.rmNode.getRackName();
-  }
-
-  /**
-   * The Scheduler has allocated containers on this node to the 
-   * given application.
-   * 
-   * @param applicationId application
-   * @param rmContainer allocated container
-   */
-  public synchronized void allocateContainer(ApplicationId applicationId, 
-      RMContainer rmContainer) {
-    Container container = rmContainer.getContainer();
-    deductAvailableResource(container.getResource());
-    ++numContainers;
-    
-    launchedContainers.put(container.getId(), rmContainer);
-
-    LOG.info("Assigned container " + container.getId() + 
-        " of capacity " + container.getResource() + " on host " + rmNode.getNodeAddress() + 
-        ", which currently has " + numContainers + " containers, " + 
-        getUsedResource() + " used and " + 
-        getAvailableResource() + " available");
-  }
-
-  @Override
-  public synchronized Resource getAvailableResource() {
-    return this.availableResource;
-  }
-
-  @Override
-  public synchronized Resource getUsedResource() {
-    return this.usedResource;
-  }
-
-  @Override
-  public Resource getTotalResource() {
-    return this.totalResourceCapability;
-  }
-
-  private synchronized boolean isValidContainer(Container c) {    
-    if (launchedContainers.containsKey(c.getId()))
-      return true;
-    return false;
-  }
-
-  private synchronized void updateResource(Container container) {
-    addAvailableResource(container.getResource());
-    --numContainers;
-  }
-  
-  /**
-   * Release an allocated container on this node.
-   * @param container container to be released
-   */
-  public synchronized void releaseContainer(Container container) {
-    if (!isValidContainer(container)) {
-      LOG.error("Invalid container released " + container);
-      return;
-    }
-
-    /* remove the containers from the nodemanger */
-    if (null != launchedContainers.remove(container.getId())) {
-      updateResource(container);
-    }
-
-    LOG.info("Released container " + container.getId() + 
-        " of capacity " + container.getResource() + " on host " + rmNode.getNodeAddress() + 
-        ", which currently has " + numContainers + " containers, " + 
-        getUsedResource() + " used and " + getAvailableResource()
-        + " available" + ", release resources=" + true);
-  }
-
-
-  private synchronized void addAvailableResource(Resource resource) {
-    if (resource == null) {
-      LOG.error("Invalid resource addition of null resource for "
-          + rmNode.getNodeAddress());
-      return;
-    }
-    Resources.addTo(availableResource, resource);
-    Resources.subtractFrom(usedResource, resource);
-  }
-
-  private synchronized void deductAvailableResource(Resource resource) {
-    if (resource == null) {
-      LOG.error("Invalid deduction of null resource for "
-          + rmNode.getNodeAddress());
-      return;
-    }
-    Resources.subtractFrom(availableResource, resource);
-    Resources.addTo(usedResource, resource);
+    super(node, usePortForNodeName);
   }
 
   @Override
-  public String toString() {
-    return "host: " + rmNode.getNodeAddress() + " #containers=" + getNumContainers() +  
-      " available=" + getAvailableResource().getMemory() + 
-      " used=" + getUsedResource().getMemory();
-  }
-
-  @Override
-  public int getNumContainers() {
-    return numContainers;
-  }
-
-  public synchronized List<RMContainer> getRunningContainers() {
-    return new ArrayList<RMContainer>(launchedContainers.values());
-  }
-
   public synchronized void reserveResource(
-      SchedulerApplicationAttempt application, Priority priority, 
-      RMContainer reservedContainer) {
+      SchedulerApplicationAttempt application, Priority priority,
+      RMContainer container) {
     // Check if it's already reserved
-    if (this.reservedContainer != null) {
+    RMContainer reservedContainer = getReservedContainer();
+    if (reservedContainer != null) {
       // Sanity check
-      if (!reservedContainer.getContainer().getNodeId().equals(getNodeID())) {
+      if (!container.getContainer().getNodeId().equals(getNodeID())) {
         throw new IllegalStateException("Trying to reserve" +
-            " container " + reservedContainer +
-            " on node " + reservedContainer.getReservedNode() + 
-            " when currently" + " reserved resource " + this.reservedContainer +
-            " on node " + this.reservedContainer.getReservedNode());
+            " container " + container +
+            " on node " + container.getReservedNode() + 
+            " when currently" + " reserved resource " + reservedContainer +
+            " on node " + reservedContainer.getReservedNode());
       }
       
       // Cannot reserve more than one application attempt on a given node!
       // Reservation is still against attempt.
-      if (!this.reservedContainer.getContainer().getId().getApplicationAttemptId().equals(
-          reservedContainer.getContainer().getId().getApplicationAttemptId())) {
+      if (!reservedContainer.getContainer().getId().getApplicationAttemptId()
+          .equals(container.getContainer().getId().getApplicationAttemptId())) {
         throw new IllegalStateException("Trying to reserve" +
-        		" container " + reservedContainer + 
+            " container " + container + 
             " for application " + application.getApplicationAttemptId() + 
             " when currently" +
-            " reserved container " + this.reservedContainer +
+            " reserved container " + reservedContainer +
             " on node " + this);
       }
 
       if (LOG.isDebugEnabled()) {
         LOG.debug("Updated reserved container "
-            + reservedContainer.getContainer().getId() + " on node " + this
+            + container.getContainer().getId() + " on node " + this
             + " for application attempt "
             + application.getApplicationAttemptId());
       }
     } else {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Reserved container "
-            + reservedContainer.getContainer().getId() + " on node " + this
+            + container.getContainer().getId() + " on node " + this
             + " for application attempt "
             + application.getApplicationAttemptId());
       }
     }
-    this.reservedContainer = reservedContainer;
+    setReservedContainer(container);
   }
 
+  @Override
   public synchronized void unreserveResource(
       SchedulerApplicationAttempt application) {
-    
+
     // adding NP checks as this can now be called for preemption
-    if (reservedContainer != null
-        && reservedContainer.getContainer() != null
-        && reservedContainer.getContainer().getId() != null
-        && reservedContainer.getContainer().getId().getApplicationAttemptId() != null) {
+    if (getReservedContainer() != null
+        && getReservedContainer().getContainer() != null
+        && getReservedContainer().getContainer().getId() != null
+        && getReservedContainer().getContainer().getId()
+          .getApplicationAttemptId() != null) {
 
       // Cannot unreserve for wrong application...
       ApplicationAttemptId reservedApplication =
-          reservedContainer.getContainer().getId().getApplicationAttemptId();
+          getReservedContainer().getContainer().getId()
+            .getApplicationAttemptId();
       if (!reservedApplication.equals(
           application.getApplicationAttemptId())) {
         throw new IllegalStateException("Trying to unreserve " +
@@ -269,17 +105,6 @@ public class FiCaSchedulerNode extends S
             " on node " + this);
       }
     }
-    reservedContainer = null;
-  }
-
-  public synchronized RMContainer getReservedContainer() {
-    return reservedContainer;
+    setReservedContainer(null);
   }
-
-  @Override
-  public synchronized void applyDeltaOnAvailableResource(Resource deltaResource) {
-    // we can only adjust available resource if total resource is changed.
-    Resources.addTo(this.availableResource, deltaResource);
-  }
-
 }

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java Thu May 22 05:32:26 2014
@@ -119,9 +119,9 @@ public abstract class FSQueue extends Sc
     // TODO: we might change these queue metrics around a little bit
     // to match the semantics of the fair scheduler.
     queueInfo.setCapacity((float) getFairShare().getMemory() /
-        scheduler.getClusterCapacity().getMemory());
+        scheduler.getClusterResource().getMemory());
     queueInfo.setCapacity((float) getResourceUsage().getMemory() /
-        scheduler.getClusterCapacity().getMemory());
+        scheduler.getClusterResource().getMemory());
     
     ArrayList<QueueInfo> childQueueInfos = new ArrayList<QueueInfo>();
     if (includeChildQueues) {

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java?rev=1596753&r1=1596752&r2=1596753&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java Thu May 22 05:32:26 2014
@@ -18,28 +18,16 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
-import org.apache.hadoop.yarn.util.resource.Resources;
 
 @Private
 @Unstable
@@ -47,208 +35,56 @@ public class FSSchedulerNode extends Sch
 
   private static final Log LOG = LogFactory.getLog(FSSchedulerNode.class);
 
-  private static final RecordFactory recordFactory = RecordFactoryProvider
-      .getRecordFactory(null);
-
-  private Resource availableResource;
-  private Resource usedResource = recordFactory.newRecordInstance(Resource.class);
-  private Resource totalResourceCapability;
-
-  private volatile int numContainers;
-
-  private RMContainer reservedContainer;
   private AppSchedulable reservedAppSchedulable;
-  
-  /* set of containers that are allocated containers */
-  private final Map<ContainerId, RMContainer> launchedContainers = 
-    new HashMap<ContainerId, RMContainer>();
-  
-  private final RMNode rmNode;
-  private final String nodeName;
 
   public FSSchedulerNode(RMNode node, boolean usePortForNodeName) {
-    this.rmNode = node;
-    this.availableResource = Resources.clone(node.getTotalCapability());
-    totalResourceCapability =
-        Resource.newInstance(node.getTotalCapability().getMemory(), node
-            .getTotalCapability().getVirtualCores());
-    if (usePortForNodeName) {
-      nodeName = rmNode.getHostName() + ":" + node.getNodeID().getPort();
-    } else {
-      nodeName = rmNode.getHostName();
-    }
-  }
-
-  public RMNode getRMNode() {
-    return rmNode;
-  }
-
-  public NodeId getNodeID() {
-    return rmNode.getNodeID();
-  }
-
-  public String getHttpAddress() {
-    return rmNode.getHttpAddress();
-  }
-
-  @Override
-  public String getNodeName() {
-    return nodeName;
+    super(node, usePortForNodeName);
   }
 
   @Override
-  public String getRackName() {
-    return rmNode.getRackName();
-  }
-
-  /**
-   * The Scheduler has allocated containers on this node to the 
-   * given application.
-   * 
-   * @param applicationId application
-   * @param rmContainer allocated container
-   */
-  public synchronized void allocateContainer(ApplicationId applicationId, 
-      RMContainer rmContainer) {
-    Container container = rmContainer.getContainer();
-    deductAvailableResource(container.getResource());
-    ++numContainers;
-    
-    launchedContainers.put(container.getId(), rmContainer);
-
-    LOG.info("Assigned container " + container.getId() + 
-        " of capacity " + container.getResource() + " on host " + rmNode.getNodeAddress() + 
-        ", which currently has " + numContainers + " containers, " + 
-        getUsedResource() + " used and " + 
-        getAvailableResource() + " available");
-  }
-
-  @Override
-  public synchronized Resource getAvailableResource() {
-    return availableResource;
-  }
-
-  @Override
-  public synchronized Resource getUsedResource() {
-    return usedResource;
-  }
-
-  private synchronized boolean isValidContainer(Container c) {    
-    if (launchedContainers.containsKey(c.getId())) {
-      return true;
-    }
-    return false;
-  }
-
-  private synchronized void updateResource(Container container) {
-    addAvailableResource(container.getResource());
-    --numContainers;
-  }
-  
-  /**
-   * Release an allocated container on this node.
-   * @param container container to be released
-   */
-  public synchronized void releaseContainer(Container container) {
-    if (!isValidContainer(container)) {
-      LOG.error("Invalid container released " + container);
-      return;
-    }
-
-    /* remove the containers from the nodemanger */
-    launchedContainers.remove(container.getId());
-    updateResource(container);
-
-    LOG.info("Released container " + container.getId() + 
-        " of capacity " + container.getResource() + " on host " + rmNode.getNodeAddress() + 
-        ", which currently has " + numContainers + " containers, " + 
-        getUsedResource() + " used and " + getAvailableResource()
-        + " available" + ", release resources=" + true);
-  }
-
-
-  private synchronized void addAvailableResource(Resource resource) {
-    if (resource == null) {
-      LOG.error("Invalid resource addition of null resource for "
-          + rmNode.getNodeAddress());
-      return;
-    }
-    Resources.addTo(availableResource, resource);
-    Resources.subtractFrom(usedResource, resource);
-  }
-
-  @Override
-  public Resource getTotalResource() {
-    return this.totalResourceCapability;
-  }
-
-  private synchronized void deductAvailableResource(Resource resource) {
-    if (resource == null) {
-      LOG.error("Invalid deduction of null resource for "
-          + rmNode.getNodeAddress());
-      return;
-    }
-    Resources.subtractFrom(availableResource, resource);
-    Resources.addTo(usedResource, resource);
-  }
-
-  @Override
-  public String toString() {
-    return "host: " + rmNode.getNodeAddress() + " #containers=" + getNumContainers() +  
-      " available=" + getAvailableResource() + 
-      " used=" + getUsedResource();
-  }
-
-  @Override
-  public int getNumContainers() {
-    return numContainers;
-  }
-
-  public synchronized List<RMContainer> getRunningContainers() {
-    return new ArrayList<RMContainer>(launchedContainers.values());
-  }
-
   public synchronized void reserveResource(
-      FSSchedulerApp application, Priority priority, 
-      RMContainer reservedContainer) {
+      SchedulerApplicationAttempt application, Priority priority,
+      RMContainer container) {
     // Check if it's already reserved
-    if (this.reservedContainer != null) {
+    RMContainer reservedContainer = getReservedContainer();
+    if (reservedContainer != null) {
       // Sanity check
-      if (!reservedContainer.getContainer().getNodeId().equals(getNodeID())) {
+      if (!container.getContainer().getNodeId().equals(getNodeID())) {
         throw new IllegalStateException("Trying to reserve" +
-            " container " + reservedContainer +
-            " on node " + reservedContainer.getReservedNode() + 
-            " when currently" + " reserved resource " + this.reservedContainer +
-            " on node " + this.reservedContainer.getReservedNode());
+            " container " + container +
+            " on node " + container.getReservedNode() + 
+            " when currently" + " reserved resource " + reservedContainer +
+            " on node " + reservedContainer.getReservedNode());
       }
       
       // Cannot reserve more than one application on a given node!
-      if (!this.reservedContainer.getContainer().getId().getApplicationAttemptId().equals(
-          reservedContainer.getContainer().getId().getApplicationAttemptId())) {
+      if (!reservedContainer.getContainer().getId().getApplicationAttemptId()
+          .equals(container.getContainer().getId().getApplicationAttemptId())) {
         throw new IllegalStateException("Trying to reserve" +
-        		" container " + reservedContainer + 
+            " container " + container + 
             " for application " + application.getApplicationId() + 
             " when currently" +
-            " reserved container " + this.reservedContainer +
+            " reserved container " + reservedContainer +
             " on node " + this);
       }
 
       LOG.info("Updated reserved container " + 
-          reservedContainer.getContainer().getId() + " on node " + 
+          container.getContainer().getId() + " on node " + 
           this + " for application " + application);
     } else {
-      LOG.info("Reserved container " + reservedContainer.getContainer().getId() + 
+      LOG.info("Reserved container " + container.getContainer().getId() + 
           " on node " + this + " for application " + application);
     }
-    this.reservedContainer = reservedContainer;
-    this.reservedAppSchedulable = application.getAppSchedulable();
+    setReservedContainer(container);
+    this.reservedAppSchedulable = ((FSSchedulerApp) application).getAppSchedulable();
   }
 
+  @Override
   public synchronized void unreserveResource(
-      FSSchedulerApp application) {
+      SchedulerApplicationAttempt application) {
     // Cannot unreserve for wrong application...
     ApplicationAttemptId reservedApplication = 
-        reservedContainer.getContainer().getId().getApplicationAttemptId(); 
+        getReservedContainer().getContainer().getId().getApplicationAttemptId(); 
     if (!reservedApplication.equals(
         application.getApplicationAttemptId())) {
       throw new IllegalStateException("Trying to unreserve " +  
@@ -258,22 +94,11 @@ public class FSSchedulerNode extends Sch
           " on node " + this);
     }
     
-    this.reservedContainer = null;
+    setReservedContainer(null);
     this.reservedAppSchedulable = null;
   }
 
-  public synchronized RMContainer getReservedContainer() {
-    return reservedContainer;
-  }
-
   public synchronized AppSchedulable getReservedAppSchedulable() {
     return reservedAppSchedulable;
   }
-  
-  @Override
-  public synchronized void applyDeltaOnAvailableResource(Resource deltaResource) {
-    // we can only adjust available resource if total resource is changed.
-    Resources.addTo(this.availableResource, deltaResource);
-  }
-  
 }