YARN-5821. Drop left-over preemption-related code and clean up method 
visibilities in the Schedulable hierarchy
(Contributed by Karthik Kambatla via Daniel Templeton)


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

Branch: refs/heads/YARN-4752
Commit: e3b5c489bd96e51928f15604e2f99b21cf590997
Parents: 07b9bf3
Author: Daniel Templeton <templ...@apache.org>
Authored: Thu Nov 3 14:50:09 2016 -0700
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Thu Nov 10 14:48:15 2016 -0800

----------------------------------------------------------------------
 .../scheduler/fair/FSAppAttempt.java            |  77 ++++----------
 .../scheduler/fair/FSLeafQueue.java             | 102 +++----------------
 .../scheduler/fair/FSParentQueue.java           |  46 ++-------
 .../resourcemanager/scheduler/fair/FSQueue.java |  20 ++--
 .../scheduler/fair/Schedulable.java             |  29 +++---
 .../scheduler/fair/FakeSchedulable.java         |   5 -
 .../scheduler/fair/TestSchedulingPolicy.java    |   5 -
 7 files changed, 63 insertions(+), 221 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3b5c489/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.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/fair/FSAppAttempt.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
index d9fdaba..8b4627e 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
@@ -18,18 +18,17 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
-import java.io.Serializable;
 import java.text.DecimalFormat;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
-import java.util.Comparator;
+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 static com.sun.xml.internal.xsom.impl.UName.comparator;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -81,7 +80,6 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   private FairScheduler scheduler;
   private FSQueue fsQueue;
   private Resource fairShare = Resources.createResource(0, 0);
-  private RMContainerComparator comparator = new RMContainerComparator();
 
   // Preemption related variables
   private Resource fairshareStarvation = Resources.none();
@@ -121,7 +119,7 @@ public class FSAppAttempt extends 
SchedulerApplicationAttempt
     this.resourceWeights = new ResourceWeights();
   }
 
-  public ResourceWeights getResourceWeights() {
+  ResourceWeights getResourceWeights() {
     return resourceWeights;
   }
 
@@ -132,7 +130,7 @@ public class FSAppAttempt extends 
SchedulerApplicationAttempt
     return queue.getMetrics();
   }
 
-  public void containerCompleted(RMContainer rmContainer,
+  void containerCompleted(RMContainer rmContainer,
       ContainerStatus containerStatus, RMContainerEventType event) {
     try {
       writeLock.lock();
@@ -491,7 +489,7 @@ public class FSAppAttempt extends 
SchedulerApplicationAttempt
    * @param schedulerKey Scheduler Key
    * @param level NodeType
    */
-  public void resetAllowedLocalityLevel(
+  void resetAllowedLocalityLevel(
       SchedulerRequestKey schedulerKey, NodeType level) {
     NodeType old;
     try {
@@ -513,45 +511,45 @@ public class FSAppAttempt extends 
SchedulerApplicationAttempt
   }
 
   // Preemption related methods
-  public Resource getStarvation() {
+  Resource getStarvation() {
     return Resources.add(fairshareStarvation, minshareStarvation);
   }
 
-  public void setMinshareStarvation(Resource starvation) {
+  void setMinshareStarvation(Resource starvation) {
     this.minshareStarvation = starvation;
   }
 
-  public void resetMinshareStarvation() {
+  void resetMinshareStarvation() {
     this.minshareStarvation = Resources.none();
   }
 
-  public void addPreemption(RMContainer container) {
+  void addPreemption(RMContainer container) {
     containersToPreempt.add(container);
     Resources.addTo(preemptedResources, container.getAllocatedResource());
   }
 
-  public Set<RMContainer> getPreemptionContainers() {
+  Set<RMContainer> getPreemptionContainers() {
     return containersToPreempt;
   }
   
 
-  public Resource getPreemptedResources() {
+  private Resource getPreemptedResources() {
     return preemptedResources;
   }
 
-  public void resetPreemptedResources() {
+  void resetPreemptedResources() {
     preemptedResources = Resources.createResource(0);
     for (RMContainer container : getPreemptionContainers()) {
       Resources.addTo(preemptedResources, container.getAllocatedResource());
     }
   }
 
-  public void clearPreemptedResources() {
+  void clearPreemptedResources() {
     preemptedResources.setMemorySize(0);
     preemptedResources.setVirtualCores(0);
   }
 
-  public boolean canContainerBePreempted(RMContainer container) {
+  boolean canContainerBePreempted(RMContainer container) {
     // Sanity check that the app owns this container
     if (!getLiveContainersMap().containsKey(container.getContainerId()) &&
         !newlyAllocatedContainers.contains(container)) {
@@ -585,7 +583,7 @@ public class FSAppAttempt extends 
SchedulerApplicationAttempt
    * @param schedulerKey Scheduler Key
    * @return Container
    */
-  public Container createContainer(FSSchedulerNode node, Resource capability,
+  private Container createContainer(FSSchedulerNode node, Resource capability,
       SchedulerRequestKey schedulerKey) {
 
     NodeId nodeId = node.getRMNode().getNodeID();
@@ -593,12 +591,10 @@ public class FSAppAttempt extends 
SchedulerApplicationAttempt
         getApplicationAttemptId(), getNewContainerId());
 
     // Create the container
-    Container container = BuilderUtils.newContainer(containerId, nodeId,
+    return BuilderUtils.newContainer(containerId, nodeId,
         node.getRMNode().getHttpAddress(), capability,
         schedulerKey.getPriority(), null,
         schedulerKey.getAllocationRequestId());
-
-    return container;
   }
 
   /**
@@ -853,7 +849,8 @@ public class FSAppAttempt extends 
SchedulerApplicationAttempt
     }
 
     Collection<SchedulerRequestKey> keysToTry = (reserved) ?
-        Arrays.asList(node.getReservedContainer().getReservedSchedulerKey()) :
+        Collections.singletonList(
+            node.getReservedContainer().getReservedSchedulerKey()) :
         getSchedulerKeys();
 
     // For each priority, see if we can schedule a node local, rack local
@@ -1011,7 +1008,7 @@ public class FSAppAttempt extends 
SchedulerApplicationAttempt
    *     Node that the application has an existing reservation on
    * @return whether the reservation on the given node is valid.
    */
-  public boolean assignReservedContainer(FSSchedulerNode node) {
+  boolean assignReservedContainer(FSSchedulerNode node) {
     RMContainer rmContainer = node.getReservedContainer();
     SchedulerRequestKey reservedSchedulerKey =
         rmContainer.getReservedSchedulerKey();
@@ -1040,19 +1037,6 @@ public class FSAppAttempt extends 
SchedulerApplicationAttempt
     return true;
   }
 
-  static class RMContainerComparator implements Comparator<RMContainer>,
-      Serializable {
-    @Override
-    public int compare(RMContainer c1, RMContainer c2) {
-      int ret = c1.getContainer().getPriority().compareTo(
-          c2.getContainer().getPriority());
-      if (ret == 0) {
-        return c2.getContainerId().compareTo(c1.getContainerId());
-      }
-      return ret;
-    }
-  }
-
   /**
    * Helper method that computes the extent of fairshare fairshareStarvation.
    */
@@ -1079,7 +1063,7 @@ public class FSAppAttempt extends 
SchedulerApplicationAttempt
     return this.fairshareStarvation;
   }
 
-  public ResourceRequest getNextResourceRequest() {
+  ResourceRequest getNextResourceRequest() {
     return appSchedulingInfo.getNextResourceRequest();
   }
 
@@ -1197,25 +1181,4 @@ public class FSAppAttempt extends 
SchedulerApplicationAttempt
     updateAMContainerDiagnostics(AMState.INACTIVATED,
         diagnosticMessageBldr.toString());
   }
-
-  /**
-   * Preempt a running container according to the priority
-   */
-  @Override
-  public RMContainer preemptContainer() {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("App " + getName() + " is going to preempt a running " +
-          "container");
-    }
-
-    RMContainer toBePreempted = null;
-    for (RMContainer container : getLiveContainers()) {
-      if (!getPreemptionContainers().contains(container) &&
-          (toBePreempted == null ||
-              comparator.compare(toBePreempted, container) > 0)) {
-        toBePreempted = container;
-      }
-    }
-    return toBePreempted;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3b5c489/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
----------------------------------------------------------------------
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/fair/FSLeafQueue.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
index 90a381e..3fcf627 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
@@ -21,7 +21,6 @@ package 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.List;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
@@ -72,7 +71,7 @@ public class FSLeafQueue extends FSQueue {
   private Resource amResourceUsage;
 
   private final ActiveUsersManager activeUsersManager;
-  public static final List<FSQueue> EMPTY_LIST = Collections.emptyList();
+  private static final List<FSQueue> EMPTY_LIST = Collections.emptyList();
 
   public FSLeafQueue(String name, FairScheduler scheduler,
       FSParentQueue parent) {
@@ -84,7 +83,7 @@ public class FSLeafQueue extends FSQueue {
     amResourceUsage = Resource.newInstance(0, 0);
   }
   
-  public void addApp(FSAppAttempt app, boolean runnable) {
+  void addApp(FSAppAttempt app, boolean runnable) {
     writeLock.lock();
     try {
       if (runnable) {
@@ -111,7 +110,7 @@ public class FSLeafQueue extends FSQueue {
    * Removes the given app from this queue.
    * @return whether or not the app was runnable
    */
-  public boolean removeApp(FSAppAttempt app) {
+  boolean removeApp(FSAppAttempt app) {
     boolean runnable = false;
 
     // Remove app from runnable/nonRunnable list while holding the write lock
@@ -142,7 +141,7 @@ public class FSLeafQueue extends FSQueue {
    * Removes the given app if it is non-runnable and belongs to this queue
    * @return true if the app is removed, false otherwise
    */
-  public boolean removeNonRunnableApp(FSAppAttempt app) {
+  boolean removeNonRunnableApp(FSAppAttempt app) {
     writeLock.lock();
     try {
       return nonRunnableApps.remove(app);
@@ -151,7 +150,7 @@ public class FSLeafQueue extends FSQueue {
     }
   }
 
-  public boolean isRunnableApp(FSAppAttempt attempt) {
+  boolean isRunnableApp(FSAppAttempt attempt) {
     readLock.lock();
     try {
       return runnableApps.contains(attempt);
@@ -160,7 +159,7 @@ public class FSLeafQueue extends FSQueue {
     }
   }
 
-  public boolean isNonRunnableApp(FSAppAttempt attempt) {
+  boolean isNonRunnableApp(FSAppAttempt attempt) {
     readLock.lock();
     try {
       return nonRunnableApps.contains(attempt);
@@ -169,30 +168,8 @@ public class FSLeafQueue extends FSQueue {
     }
   }
 
-  public void resetPreemptedResources() {
-    readLock.lock();
-    try {
-      for (FSAppAttempt attempt : runnableApps) {
-        attempt.resetPreemptedResources();
-      }
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  public void clearPreemptedResources() {
-    readLock.lock();
-    try {
-      for (FSAppAttempt attempt : runnableApps) {
-        attempt.clearPreemptedResources();
-      }
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  public List<FSAppAttempt> getCopyOfNonRunnableAppSchedulables() {
-    List<FSAppAttempt> appsToReturn = new ArrayList<FSAppAttempt>();
+  List<FSAppAttempt> getCopyOfNonRunnableAppSchedulables() {
+    List<FSAppAttempt> appsToReturn = new ArrayList<>();
     readLock.lock();
     try {
       appsToReturn.addAll(nonRunnableApps);
@@ -320,7 +297,7 @@ public class FSLeafQueue extends FSQueue {
     return usage;
   }
 
-  public Resource getAmResourceUsage() {
+  Resource getAmResourceUsage() {
     return amResourceUsage;
   }
 
@@ -407,42 +384,6 @@ public class FSLeafQueue extends FSQueue {
   }
 
   @Override
-  public RMContainer preemptContainer() {
-    RMContainer toBePreempted = null;
-
-    // If this queue is not over its fair share, reject
-    if (!preemptContainerPreCheck()) {
-      return toBePreempted;
-    }
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Queue " + getName() + " is going to preempt a container " +
-          "from its applications.");
-    }
-
-    // Choose the app that is most over fair share
-    Comparator<Schedulable> comparator = policy.getComparator();
-    FSAppAttempt candidateSched = null;
-    readLock.lock();
-    try {
-      for (FSAppAttempt sched : runnableApps) {
-        if (candidateSched == null ||
-            comparator.compare(sched, candidateSched) > 0) {
-          candidateSched = sched;
-        }
-      }
-    } finally {
-      readLock.unlock();
-    }
-
-    // Preempt from the selected app
-    if (candidateSched != null) {
-      toBePreempted = candidateSched.preemptContainer();
-    }
-    return toBePreempted;
-  }
-
-  @Override
   public List<FSQueue> getChildQueues() {
     return EMPTY_LIST;
   }
@@ -451,7 +392,7 @@ public class FSLeafQueue extends FSQueue {
   public List<QueueUserACLInfo> getQueueUserAclInfo(UserGroupInformation user) 
{
     QueueUserACLInfo userAclInfo =
       recordFactory.newRecordInstance(QueueUserACLInfo.class);
-    List<QueueACL> operations = new ArrayList<QueueACL>();
+    List<QueueACL> operations = new ArrayList<>();
     for (QueueACL operation : QueueACL.values()) {
       if (hasAccess(operation, user)) {
         operations.add(operation);
@@ -463,10 +404,6 @@ public class FSLeafQueue extends FSQueue {
     return Collections.singletonList(userAclInfo);
   }
   
-  public long getLastTimeAtMinShare() {
-    return lastTimeAtMinShare;
-  }
-
   private void setLastTimeAtMinShare(long lastTimeAtMinShare) {
     this.lastTimeAtMinShare = lastTimeAtMinShare;
   }
@@ -481,7 +418,7 @@ public class FSLeafQueue extends FSQueue {
     }
   }
 
-  public int getNumNonRunnableApps() {
+  int getNumNonRunnableApps() {
     readLock.lock();
     try {
       return nonRunnableApps.size();
@@ -533,10 +470,11 @@ public class FSLeafQueue extends FSQueue {
   /**
    * Check whether this queue can run this application master under the
    * maxAMShare limit.
-   * @param amResource
+   *
+   * @param amResource resources required to run the AM
    * @return true if this queue can run
    */
-  public boolean canRunAppAM(Resource amResource) {
+  boolean canRunAppAM(Resource amResource) {
     if (Math.abs(maxAMShare - -1.0f) < 0.0001) {
       return true;
     }
@@ -561,7 +499,7 @@ public class FSLeafQueue extends FSQueue {
     return Resources.fitsIn(ifRunAMResource, maxAMResource);
   }
 
-  public void addAMResourceUsage(Resource amResource) {
+  void addAMResourceUsage(Resource amResource) {
     if (amResource != null) {
       Resources.addTo(amResourceUsage, amResource);
     }
@@ -583,16 +521,6 @@ public class FSLeafQueue extends FSQueue {
   }
 
   /**
-   * Helper method to check if the queue should preempt containers
-   *
-   * @return true if check passes (can preempt) or false otherwise
-   */
-  private boolean preemptContainerPreCheck() {
-    return parent.getPolicy().checkIfUsageOverFairShare(getResourceUsage(),
-        getFairShare());
-  }
-
-  /**
    * Helper method to compute the amount of minshare starvation.
    *
    * @return the extent of minshare starvation

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3b5c489/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
----------------------------------------------------------------------
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/fair/FSParentQueue.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
index 9eb5983..16570aa 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
@@ -21,7 +21,6 @@ package 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.List;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
@@ -61,7 +60,7 @@ public class FSParentQueue extends FSQueue {
     super(name, scheduler, parent);
   }
   
-  public void addChildQueue(FSQueue child) {
+  void addChildQueue(FSQueue child) {
     writeLock.lock();
     try {
       childQueues.add(child);
@@ -70,7 +69,7 @@ public class FSParentQueue extends FSQueue {
     }
   }
 
-  public void removeChildQueue(FSQueue child) {
+  void removeChildQueue(FSQueue child) {
     writeLock.lock();
     try {
       childQueues.remove(child);
@@ -93,7 +92,7 @@ public class FSParentQueue extends FSQueue {
     }
   }
 
-  public void recomputeSteadyShares() {
+  void recomputeSteadyShares() {
     readLock.lock();
     try {
       policy.computeSteadyShares(childQueues, getSteadyFairShare());
@@ -188,7 +187,7 @@ public class FSParentQueue extends FSQueue {
   
   @Override
   public List<QueueUserACLInfo> getQueueUserAclInfo(UserGroupInformation user) 
{
-    List<QueueUserACLInfo> userAcls = new ArrayList<QueueUserACLInfo>();
+    List<QueueUserACLInfo> userAcls = new ArrayList<>();
     
     // Add queue acls
     userAcls.add(getUserAclInfo(user));
@@ -246,39 +245,6 @@ public class FSParentQueue extends FSQueue {
   }
 
   @Override
-  public RMContainer preemptContainer() {
-    RMContainer toBePreempted = null;
-
-    // Find the childQueue which is most over fair share
-    FSQueue candidateQueue = null;
-    Comparator<Schedulable> comparator = policy.getComparator();
-
-    readLock.lock();
-    try {
-      for (FSQueue queue : childQueues) {
-        // Skip selection for non-preemptable queue
-        if (!queue.isPreemptable()) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("skipping from queue=" + getName()
-                + " because it's a non-preemptable queue");
-          }
-        } else if (candidateQueue == null ||
-                  comparator.compare(queue, candidateQueue) > 0) {
-          candidateQueue = queue;
-        }
-      }
-    } finally {
-      readLock.unlock();
-    }
-
-    // Let the selected queue choose which of its container to preempt
-    if (candidateQueue != null) {
-      toBePreempted = candidateQueue.preemptContainer();
-    }
-    return toBePreempted;
-  }
-
-  @Override
   public List<FSQueue> getChildQueues() {
     readLock.lock();
     try {
@@ -301,7 +267,7 @@ public class FSParentQueue extends FSQueue {
     super.policy = policy;
   }
 
-  public void incrementRunnableApps() {
+  void incrementRunnableApps() {
     writeLock.lock();
     try {
       runnableApps++;
@@ -310,7 +276,7 @@ public class FSParentQueue extends FSQueue {
     }
   }
   
-  public void decrementRunnableApps() {
+  void decrementRunnableApps() {
     writeLock.lock();
     try {
       runnableApps--;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3b5c489/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
----------------------------------------------------------------------
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/fair/FSQueue.java
 
b/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
index e1e528e..572b5f9 100644
--- 
a/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
+++ 
b/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
@@ -253,7 +253,7 @@ public abstract class FSQueue implements Queue, Schedulable 
{
     return steadyFairShare;
   }
 
-  public void setSteadyFairShare(Resource steadyFairShare) {
+  void setSteadyFairShare(Resource steadyFairShare) {
     this.steadyFairShare = steadyFairShare;
     metrics.setSteadyFairShare(steadyFairShare);
   }
@@ -262,27 +262,27 @@ public abstract class FSQueue implements Queue, 
Schedulable {
     return scheduler.getAllocationConfiguration().hasAccess(name, acl, user);
   }
 
-  public long getFairSharePreemptionTimeout() {
+  long getFairSharePreemptionTimeout() {
     return fairSharePreemptionTimeout;
   }
 
-  public void setFairSharePreemptionTimeout(long fairSharePreemptionTimeout) {
+  void setFairSharePreemptionTimeout(long fairSharePreemptionTimeout) {
     this.fairSharePreemptionTimeout = fairSharePreemptionTimeout;
   }
 
-  public long getMinSharePreemptionTimeout() {
+  long getMinSharePreemptionTimeout() {
     return minSharePreemptionTimeout;
   }
 
-  public void setMinSharePreemptionTimeout(long minSharePreemptionTimeout) {
+  void setMinSharePreemptionTimeout(long minSharePreemptionTimeout) {
     this.minSharePreemptionTimeout = minSharePreemptionTimeout;
   }
 
-  public float getFairSharePreemptionThreshold() {
+  float getFairSharePreemptionThreshold() {
     return fairSharePreemptionThreshold;
   }
 
-  public void setFairSharePreemptionThreshold(float 
fairSharePreemptionThreshold) {
+  void setFairSharePreemptionThreshold(float fairSharePreemptionThreshold) {
     this.fairSharePreemptionThreshold = fairSharePreemptionThreshold;
   }
 
@@ -297,7 +297,7 @@ public abstract class FSQueue implements Queue, Schedulable 
{
    * @param checkStarvation whether to check for fairshare or minshare
    *                        starvation on update
    */
-  public abstract void updateInternal(boolean checkStarvation);
+  abstract void updateInternal(boolean checkStarvation);
 
   public void update(Resource fairShare, boolean checkStarvation) {
     setFairShare(fairShare);
@@ -355,7 +355,7 @@ public abstract class FSQueue implements Queue, Schedulable 
{
    * 
    * @return true if check passes (can assign) or false otherwise
    */
-  protected boolean assignContainerPreCheck(FSSchedulerNode node) {
+  boolean assignContainerPreCheck(FSSchedulerNode node) {
     if (!Resources.fitsIn(getResourceUsage(), maxShare)
         || node.getReservedContainer() != null) {
       return false;
@@ -411,7 +411,7 @@ public abstract class FSQueue implements Queue, Schedulable 
{
     return null;
   }
 
-  public boolean fitsInMaxShare(Resource additionalResource) {
+  boolean fitsInMaxShare(Resource additionalResource) {
     Resource usagePlusAddition =
         Resources.add(getResourceUsage(), additionalResource);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3b5c489/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.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/fair/Schedulable.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
index 289887f..cf78405 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
@@ -55,50 +55,45 @@ public interface Schedulable {
    * Name of job/queue, used for debugging as well as for breaking ties in
    * scheduling order deterministically.
    */
-  public String getName();
+  String getName();
 
   /**
    * Maximum number of resources required by this Schedulable. This is defined 
as
    * number of currently utilized resources + number of unlaunched resources 
(that
    * are either not yet launched or need to be speculated).
    */
-  public Resource getDemand();
+  Resource getDemand();
 
   /** Get the aggregate amount of resources consumed by the schedulable. */
-  public Resource getResourceUsage();
+  Resource getResourceUsage();
 
   /** Minimum Resource share assigned to the schedulable. */
-  public Resource getMinShare();
+  Resource getMinShare();
 
   /** Maximum Resource share assigned to the schedulable. */
-  public Resource getMaxShare();
+  Resource getMaxShare();
 
   /** Job/queue weight in fair sharing. */
-  public ResourceWeights getWeights();
+  ResourceWeights getWeights();
 
   /** Start time for jobs in FIFO queues; meaningless for QueueSchedulables.*/
-  public long getStartTime();
+  long getStartTime();
 
  /** Job priority for jobs in FIFO queues; meaningless for QueueSchedulables. 
*/
-  public Priority getPriority();
+  Priority getPriority();
 
   /** Refresh the Schedulable's demand and those of its children if any. */
-  public void updateDemand();
+  void updateDemand();
 
   /**
    * Assign a container on this node if possible, and return the amount of
    * resources assigned.
    */
-  public Resource assignContainer(FSSchedulerNode node);
-
-  /**
-   * Preempt a container from this Schedulable if possible.
-   */
-  public RMContainer preemptContainer();
+  Resource assignContainer(FSSchedulerNode node);
 
   /** Get the fair share assigned to this Schedulable. */
-  public Resource getFairShare();
+  Resource getFairShare();
 
   /** Assign a fair share to this Schedulable. */
-  public void setFairShare(Resource fairShare);
+  void setFairShare(Resource fairShare);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3b5c489/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FakeSchedulable.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/fair/FakeSchedulable.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FakeSchedulable.java
index 5a170cf..e802f42 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FakeSchedulable.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FakeSchedulable.java
@@ -86,11 +86,6 @@ public class FakeSchedulable implements Schedulable {
   }
 
   @Override
-  public RMContainer preemptContainer() {
-    return null;
-  }
-
-  @Override
   public Resource getFairShare() {
     return this.fairShare;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3b5c489/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.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/fair/TestSchedulingPolicy.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
index dea2dd1..57c7301 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
@@ -330,11 +330,6 @@ public class TestSchedulingPolicy {
       }
 
       @Override
-      public RMContainer preemptContainer() {
-        throw new UnsupportedOperationException();
-      }
-
-      @Override
       public Resource getFairShare() {
         throw new UnsupportedOperationException();
       }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-commits-h...@hadoop.apache.org

Reply via email to