瀏覽代碼

YARN-1241. In Fair Scheduler, maxRunningApps does not work for non-leaf queues. (Sandy Ryza)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1546626 13f79535-47bb-0310-9956-ffa450edef68
Sanford Ryza 11 年之前
父節點
當前提交
ce913ffd49
共有 15 個文件被更改,包括 777 次插入139 次删除
  1. 3 0
      hadoop-yarn-project/CHANGES.txt
  2. 1 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AppSchedulable.java
  3. 75 36
      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
  4. 14 0
      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
  5. 10 0
      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
  6. 6 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerApp.java
  7. 30 54
      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
  8. 302 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/MaxRunningAppsEnforcer.java
  9. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
  10. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerLeafQueueInfo.java
  11. 8 7
      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
  12. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSSchedulerApp.java
  13. 167 16
      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
  14. 152 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestMaxRunningAppsEnforcer.java
  15. 2 1
      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

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

@@ -108,6 +108,9 @@ Release 2.3.0 - UNRELEASED
     YARN-1239. Modified ResourceManager state-store implementations to start
     storing version numbers. (Jian He via vinodkv)
 
+    YARN-1241. In Fair Scheduler, maxRunningApps does not work for non-leaf
+    queues. (Sandy Ryza)
+
   OPTIMIZATIONS
 
   BUG FIXES

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

@@ -51,7 +51,6 @@ public class AppSchedulable extends Schedulable {
   private FairScheduler scheduler;
   private FSSchedulerApp app;
   private Resource demand = Resources.createResource(0);
-  private boolean runnable = false; // everyone starts as not runnable
   private long startTime;
   private static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
   private static final Log LOG = LogFactory.getLog(AppSchedulable.class);
@@ -61,7 +60,7 @@ public class AppSchedulable extends Schedulable {
   public AppSchedulable(FairScheduler scheduler, FSSchedulerApp app, FSLeafQueue queue) {
     this.scheduler = scheduler;
     this.app = app;
-    this.startTime = System.currentTimeMillis();
+    this.startTime = scheduler.getClock().getTime();
     this.queue = queue;
     this.containerTokenSecretManager = scheduler.
     		getContainerTokenSecretManager();
@@ -138,18 +137,6 @@ public class AppSchedulable extends Schedulable {
     return p;
   }
 
-  /**
-   * Is this application runnable? Runnable means that the user and queue
-   * application counts are within configured quotas.
-   */
-  public boolean getRunnable() {
-    return runnable;
-  }
-
-  public void setRunnable(boolean runnable) {
-    this.runnable = runnable;
-  }
-
   /**
    * Create and return a container object reflecting an allocation for the
    * given appliction on the given node with the given capability and
@@ -281,9 +268,6 @@ public class AppSchedulable extends Schedulable {
         unreserve(priority, node);
         return Resources.none();
       }
-    } else {
-      // If this app is over quota, don't schedule anything
-      if (!(getRunnable())) { return Resources.none(); }
     }
 
     Collection<Priority> prioritiesToTry = (reserved) ? 

+ 75 - 36
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

@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
-import java.util.Iterator;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -42,7 +41,9 @@ public class FSLeafQueue extends FSQueue {
   private static final Log LOG = LogFactory.getLog(
       FSLeafQueue.class.getName());
     
-  private final List<AppSchedulable> appScheds = 
+  private final List<AppSchedulable> runnableAppScheds = // apps that are runnable
+      new ArrayList<AppSchedulable>();
+  private final List<AppSchedulable> nonRunnableAppScheds =
       new ArrayList<AppSchedulable>();
   
   private final FairScheduler scheduler;
@@ -62,29 +63,51 @@ public class FSLeafQueue extends FSQueue {
     this.lastTimeAtHalfFairShare = scheduler.getClock().getTime();
   }
   
-  public void addApp(FSSchedulerApp app) {
+  public void addApp(FSSchedulerApp app, boolean runnable) {
     AppSchedulable appSchedulable = new AppSchedulable(scheduler, app, this);
     app.setAppSchedulable(appSchedulable);
-    appScheds.add(appSchedulable);
+    if (runnable) {
+      runnableAppScheds.add(appSchedulable);
+    } else {
+      nonRunnableAppScheds.add(appSchedulable);
+    }
   }
   
   // for testing
   void addAppSchedulable(AppSchedulable appSched) {
-    appScheds.add(appSched);
+    runnableAppScheds.add(appSched);
   }
   
-  public void removeApp(FSSchedulerApp app) {
-    for (Iterator<AppSchedulable> it = appScheds.iterator(); it.hasNext();) {
-      AppSchedulable appSched = it.next();
-      if (appSched.getApp() == app) {
-        it.remove();
-        break;
-      }
+  /**
+   * Removes the given app from this queue.
+   * @return whether or not the app was runnable
+   */
+  public boolean removeApp(FSSchedulerApp app) {
+    if (runnableAppScheds.remove(app.getAppSchedulable())) {
+      return true;
+    } else if (nonRunnableAppScheds.remove(app.getAppSchedulable())) {
+      return false;
+    } else {
+      throw new IllegalStateException("Given app to remove " + app +
+          " does not exist in queue " + this);
     }
   }
   
-  public Collection<AppSchedulable> getAppSchedulables() {
-    return appScheds;
+  public void makeAppRunnable(AppSchedulable appSched) {
+    if (!nonRunnableAppScheds.remove(appSched)) {
+      throw new IllegalStateException("Can't make app runnable that does not " +
+      		"already exist in queue as non-runnable" + appSched);
+    }
+    
+    runnableAppScheds.add(appSched);
+  }
+  
+  public Collection<AppSchedulable> getRunnableAppSchedulables() {
+    return runnableAppScheds;
+  }
+  
+  public List<AppSchedulable> getNonRunnableAppSchedulables() {
+    return nonRunnableAppScheds;
   }
 
   @Override
@@ -98,7 +121,7 @@ public class FSLeafQueue extends FSQueue {
   
   @Override
   public void recomputeShares() {
-    policy.computeShares(getAppSchedulables(), getFairShare());
+    policy.computeShares(getRunnableAppSchedulables(), getFairShare());
   }
 
   @Override
@@ -109,7 +132,10 @@ public class FSLeafQueue extends FSQueue {
   @Override
   public Resource getResourceUsage() {
     Resource usage = Resources.createResource(0);
-    for (AppSchedulable app : appScheds) {
+    for (AppSchedulable app : runnableAppScheds) {
+      Resources.addTo(usage, app.getResourceUsage());
+    }
+    for (AppSchedulable app : nonRunnableAppScheds) {
       Resources.addTo(usage, app.getResourceUsage());
     }
     return usage;
@@ -121,25 +147,35 @@ public class FSLeafQueue extends FSQueue {
     // Limit demand to maxResources
     Resource maxRes = queueMgr.getMaxResources(getName());
     demand = Resources.createResource(0);
-    for (AppSchedulable sched : appScheds) {
-      sched.updateDemand();
-      Resource toAdd = sched.getDemand();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Counting resource from " + sched.getName() + " " + toAdd
-            + "; Total resource consumption for " + getName() + " now "
-            + demand);
+    for (AppSchedulable sched : runnableAppScheds) {
+      if (Resources.equals(demand, maxRes)) {
+        break;
       }
-      demand = Resources.add(demand, toAdd);
-      demand = Resources.componentwiseMin(demand, maxRes);
+      updateDemandForApp(sched, maxRes);
+    }
+    for (AppSchedulable sched : nonRunnableAppScheds) {
       if (Resources.equals(demand, maxRes)) {
         break;
       }
+      updateDemandForApp(sched, maxRes);
     }
     if (LOG.isDebugEnabled()) {
       LOG.debug("The updated demand for " + getName() + " is " + demand
           + "; the max is " + maxRes);
     }
   }
+  
+  private void updateDemandForApp(AppSchedulable sched, Resource maxRes) {
+    sched.updateDemand();
+    Resource toAdd = sched.getDemand();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Counting resource from " + sched.getName() + " " + toAdd
+          + "; Total resource consumption for " + getName() + " now "
+          + demand);
+    }
+    demand = Resources.add(demand, toAdd);
+    demand = Resources.componentwiseMin(demand, maxRes);
+  }
 
   @Override
   public Resource assignContainer(FSSchedulerNode node) {
@@ -153,17 +189,15 @@ public class FSLeafQueue extends FSQueue {
     }
 
     Comparator<Schedulable> comparator = policy.getComparator();
-    Collections.sort(appScheds, comparator);
-    for (AppSchedulable sched : appScheds) {
-      if (sched.getRunnable()) {
-        if (SchedulerAppUtils.isBlacklisted(sched.getApp(), node, LOG)) {
-          continue;
-        }
-
-        assigned = sched.assignContainer(node);
-        if (!assigned.equals(Resources.none())) {
-          break;
-        }
+    Collections.sort(runnableAppScheds, comparator);
+    for (AppSchedulable sched : runnableAppScheds) {
+      if (SchedulerAppUtils.isBlacklisted(sched.getApp(), node, LOG)) {
+        continue;
+      }
+
+      assigned = sched.assignContainer(node);
+      if (!assigned.equals(Resources.none())) {
+        break;
       }
     }
     return assigned;
@@ -205,4 +239,9 @@ public class FSLeafQueue extends FSQueue {
   public void setLastTimeAtHalfFairShare(long lastTimeAtHalfFairShare) {
     this.lastTimeAtHalfFairShare = lastTimeAtHalfFairShare;
   }
+
+  @Override
+  public int getNumRunnableApps() {
+    return runnableAppScheds.size();
+  }
 }

+ 14 - 0
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

@@ -43,6 +43,7 @@ public class FSParentQueue extends FSQueue {
       new ArrayList<FSQueue>();
   private final QueueManager queueMgr;
   private Resource demand = Resources.createResource(0);
+  private int runnableApps;
   
   public FSParentQueue(String name, QueueManager queueMgr, FairScheduler scheduler,
       FSParentQueue parent) {
@@ -171,4 +172,17 @@ public class FSParentQueue extends FSQueue {
     }
     super.policy = policy;
   }
+  
+  public void incrementRunnableApps() {
+    runnableApps++;
+  }
+  
+  public void decrementRunnableApps() {
+    runnableApps--;
+  }
+
+  @Override
+  public int getNumRunnableApps() {
+    return runnableApps;
+  }
 }

+ 10 - 0
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

@@ -72,6 +72,10 @@ public abstract class FSQueue extends Schedulable implements Queue {
   public SchedulingPolicy getPolicy() {
     return policy;
   }
+  
+  public FSParentQueue getParent() {
+    return parent;
+  }
 
   protected void throwPolicyDoesnotApplyException(SchedulingPolicy policy)
       throws AllocationConfigurationException {
@@ -164,6 +168,12 @@ public abstract class FSQueue extends Schedulable implements Queue {
    */
   public abstract Collection<FSQueue> getChildQueues();
 
+  /**
+   * Return the number of apps for which containers can be allocated.
+   * Includes apps in subqueues.
+   */
+  public abstract int getNumRunnableApps();
+  
   /**
    * Helper method to check if the queue should attempt assigning resources
    * 

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

@@ -44,7 +44,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerFini
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -62,7 +61,7 @@ public class FSSchedulerApp extends SchedulerApplication {
   final Map<RMContainer, Long> preemptionMap = new HashMap<RMContainer, Long>();
   
   public FSSchedulerApp(ApplicationAttemptId applicationAttemptId, 
-      String user, Queue queue, ActiveUsersManager activeUsersManager,
+      String user, FSLeafQueue queue, ActiveUsersManager activeUsersManager,
       RMContext rmContext) {
     super(applicationAttemptId, user, queue, activeUsersManager, rmContext);
   }
@@ -327,4 +326,9 @@ public class FSSchedulerApp extends SchedulerApplication {
   public Set<RMContainer> getPreemptionContainers() {
     return preemptionMap.keySet();
   }
+  
+  @Override
+  public FSLeafQueue getQueue() {
+    return (FSLeafQueue)super.getQueue();
+  }
 }

+ 30 - 54
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

@@ -190,9 +190,13 @@ public class FairScheduler implements ResourceScheduler {
                                     // heartbeat
   protected int maxAssign; // Max containers to assign per heartbeat
 
+  @VisibleForTesting
+  final MaxRunningAppsEnforcer maxRunningEnforcer;
+  
   public FairScheduler() {
     clock = new SystemClock();
     queueMgr = new QueueManager(this);
+    maxRunningEnforcer = new MaxRunningAppsEnforcer(queueMgr);
   }
 
   private void validateConf(Configuration conf) {
@@ -272,7 +276,6 @@ public class FairScheduler implements ResourceScheduler {
    */
   protected synchronized void update() {
     queueMgr.reloadAllocsIfNecessary(); // Relaod alloc file
-    updateRunnability(); // Set job runnability based on user/queue limits
     updatePreemptionVariables(); // Determine if any queues merit preemption
 
     FSQueue rootQueue = queueMgr.getRootQueue();
@@ -377,7 +380,7 @@ public class FairScheduler implements ResourceScheduler {
     for (FSLeafQueue sched : scheds) {
       if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
           sched.getResourceUsage(), sched.getFairShare())) {
-        for (AppSchedulable as : sched.getAppSchedulables()) {
+        for (AppSchedulable as : sched.getRunnableAppSchedulables()) {
           for (RMContainer c : as.getApp().getLiveContainers()) {
             runningContainers.add(c);
             apps.put(c, as.getApp());
@@ -505,63 +508,23 @@ public class FairScheduler implements ResourceScheduler {
     return resToPreempt;
   }
 
-  /**
-   * This updates the runnability of all apps based on whether or not any
-   * users/queues have exceeded their capacity.
-   */
-  private void updateRunnability() {
-    List<AppSchedulable> apps = new ArrayList<AppSchedulable>();
-
-    // Start by marking everything as not runnable
-    for (FSLeafQueue leafQueue : queueMgr.getLeafQueues()) {
-      for (AppSchedulable a : leafQueue.getAppSchedulables()) {
-        a.setRunnable(false);
-        apps.add(a);
-      }
-    }
-    // Create a list of sorted jobs in order of start time and priority
-    Collections.sort(apps, new FifoAppComparator());
-    // Mark jobs as runnable in order of start time and priority, until
-    // user or queue limits have been reached.
-    Map<String, Integer> userApps = new HashMap<String, Integer>();
-    Map<String, Integer> queueApps = new HashMap<String, Integer>();
-
-    for (AppSchedulable app : apps) {
-      String user = app.getApp().getUser();
-      String queue = app.getApp().getQueueName();
-      int userCount = userApps.containsKey(user) ? userApps.get(user) : 0;
-      int queueCount = queueApps.containsKey(queue) ? queueApps.get(queue) : 0;
-      if (userCount < queueMgr.getUserMaxApps(user) &&
-          queueCount < queueMgr.getQueueMaxApps(queue)) {
-        userApps.put(user, userCount + 1);
-        queueApps.put(queue, queueCount + 1);
-        app.setRunnable(true);
-      }
-    }
-  }
-
   public RMContainerTokenSecretManager getContainerTokenSecretManager() {
     return rmContext.getContainerTokenSecretManager();
   }
 
   // synchronized for sizeBasedWeight
   public synchronized ResourceWeights getAppWeight(AppSchedulable app) {
-    if (!app.getRunnable()) {
-      // Job won't launch tasks, but don't return 0 to avoid division errors
-      return ResourceWeights.NEUTRAL;
-    } else {
-      double weight = 1.0;
-      if (sizeBasedWeight) {
-        // Set weight based on current memory demand
-        weight = Math.log1p(app.getDemand().getMemory()) / Math.log(2);
-      }
-      weight *= app.getPriority().getPriority();
-      if (weightAdjuster != null) {
-        // Run weight through the user-supplied weightAdjuster
-        weight = weightAdjuster.adjustWeight(app, weight);
-      }
-      return new ResourceWeights((float)weight);
+    double weight = 1.0;
+    if (sizeBasedWeight) {
+      // Set weight based on current memory demand
+      weight = Math.log1p(app.getDemand().getMemory()) / Math.log(2);
     }
+    weight *= app.getPriority().getPriority();
+    if (weightAdjuster != null) {
+      // Run weight through the user-supplied weightAdjuster
+      weight = weightAdjuster.adjustWeight(app, weight);
+    }
+    return new ResourceWeights((float)weight);
   }
 
   @Override
@@ -662,7 +625,14 @@ public class FairScheduler implements ResourceScheduler {
       return;
     }
 
-    queue.addApp(schedulerApp);
+    boolean runnable = maxRunningEnforcer.canAppBeRunnable(queue, user);
+    queue.addApp(schedulerApp, runnable);
+    if (runnable) {
+      maxRunningEnforcer.trackRunnableApp(schedulerApp);
+    } else {
+      maxRunningEnforcer.trackNonRunnableApp(schedulerApp);
+    }
+    
     queue.getMetrics().submitApp(user, applicationAttemptId.getAttemptId());
 
     applications.put(applicationAttemptId, schedulerApp);
@@ -736,8 +706,14 @@ public class FairScheduler implements ResourceScheduler {
     // Inform the queue
     FSLeafQueue queue = queueMgr.getLeafQueue(application.getQueue()
         .getQueueName(), false);
-    queue.removeApp(application);
+    boolean wasRunnable = queue.removeApp(application);
 
+    if (wasRunnable) {
+      maxRunningEnforcer.updateRunnabilityOnAppRemoval(application);
+    } else {
+      maxRunningEnforcer.untrackNonRunnableApp(application);
+    }
+    
     // Remove from our data-structure
     applications.remove(applicationAttemptId);
   }

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

@@ -0,0 +1,302 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
+
+/**
+ * Handles tracking and enforcement for user and queue maxRunningApps
+ * constraints
+ */
+public class MaxRunningAppsEnforcer {
+  private final QueueManager queueMgr;
+
+  // Tracks the number of running applications by user.
+  private final Map<String, Integer> usersNumRunnableApps;
+  @VisibleForTesting
+  final ListMultimap<String, AppSchedulable> usersNonRunnableApps;
+
+  public MaxRunningAppsEnforcer(QueueManager queueMgr) {
+    this.queueMgr = queueMgr;
+    this.usersNumRunnableApps = new HashMap<String, Integer>();
+    this.usersNonRunnableApps = ArrayListMultimap.create();
+  }
+
+  /**
+   * Checks whether making the application runnable would exceed any
+   * maxRunningApps limits.
+   */
+  public boolean canAppBeRunnable(FSQueue queue, String user) {
+    Integer userNumRunnable = usersNumRunnableApps.get(user);
+    if (userNumRunnable == null) {
+      userNumRunnable = 0;
+    }
+    if (userNumRunnable >= queueMgr.getUserMaxApps(user)) {
+      return false;
+    }
+    // Check queue and all parent queues
+    while (queue != null) {
+      int queueMaxApps = queueMgr.getQueueMaxApps(queue.getName());
+      if (queue.getNumRunnableApps() >= queueMaxApps) {
+        return false;
+      }
+      queue = queue.getParent();
+    }
+
+    return true;
+  }
+
+  /**
+   * Tracks the given new runnable app for purposes of maintaining max running
+   * app limits.
+   */
+  public void trackRunnableApp(FSSchedulerApp app) {
+    String user = app.getUser();
+    FSLeafQueue queue = app.getQueue();
+    // Increment running counts for all parent queues
+    FSParentQueue parent = queue.getParent();
+    while (parent != null) {
+      parent.incrementRunnableApps();
+      parent = parent.getParent();
+    }
+
+    Integer userNumRunnable = usersNumRunnableApps.get(user);
+    usersNumRunnableApps.put(user, (userNumRunnable == null ? 0
+        : userNumRunnable) + 1);
+  }
+
+  /**
+   * Tracks the given new non runnable app so that it can be made runnable when
+   * it would not violate max running app limits.
+   */
+  public void trackNonRunnableApp(FSSchedulerApp app) {
+    String user = app.getUser();
+    usersNonRunnableApps.put(user, app.getAppSchedulable());
+  }
+
+  /**
+   * Updates the relevant tracking variables after a runnable app with the given
+   * queue and user has been removed. Checks to see whether any other applications
+   * are now runnable and makes them so.
+   * 
+   * Runs in O(n log(n)) where n is the number of queues that are under the
+   * highest queue that went from having no slack to having slack.
+   */
+  public void updateRunnabilityOnAppRemoval(FSSchedulerApp app) {
+    // Update usersRunnableApps
+    String user = app.getUser();
+    int newUserNumRunning = usersNumRunnableApps.get(user) - 1;
+    if (newUserNumRunning == 0) {
+      usersNumRunnableApps.remove(user);
+    } else {
+      usersNumRunnableApps.put(user, newUserNumRunning);
+    }
+
+    // Update runnable app bookkeeping for queues:
+    // childqueueX might have no pending apps itself, but if a queue higher up
+    // in the hierarchy parentqueueY has a maxRunningApps set, an app completion
+    // in childqueueX could allow an app in some other distant child of
+    // parentqueueY to become runnable.
+    // An app removal will only possibly allow another app to become runnable if
+    // the queue was already at its max before the removal.
+    // Thus we find the ancestor queue highest in the tree for which the app
+    // that was at its maxRunningApps before the removal.
+    FSLeafQueue queue = app.getQueue();
+    FSQueue highestQueueWithAppsNowRunnable = (queue.getNumRunnableApps() ==
+        queueMgr.getQueueMaxApps(queue.getName()) - 1) ? queue : null;
+    FSParentQueue parent = queue.getParent();
+    while (parent != null) {
+      if (parent.getNumRunnableApps() == queueMgr.getQueueMaxApps(parent
+          .getName())) {
+        highestQueueWithAppsNowRunnable = parent;
+      }
+      parent.decrementRunnableApps();
+      parent = parent.getParent();
+    }
+
+    List<List<AppSchedulable>> appsNowMaybeRunnable =
+        new ArrayList<List<AppSchedulable>>();
+
+    // Compile lists of apps which may now be runnable
+    // We gather lists instead of building a set of all non-runnable apps so
+    // that this whole operation can be O(number of queues) instead of
+    // O(number of apps)
+    if (highestQueueWithAppsNowRunnable != null) {
+      gatherPossiblyRunnableAppLists(highestQueueWithAppsNowRunnable,
+          appsNowMaybeRunnable);
+    }
+    if (newUserNumRunning == queueMgr.getUserMaxApps(user) - 1) {
+      List<AppSchedulable> userWaitingApps = usersNonRunnableApps.get(user);
+      if (userWaitingApps != null) {
+        appsNowMaybeRunnable.add(userWaitingApps);
+      }
+    }
+
+    // Scan through and check whether this means that any apps are now runnable
+    Iterator<FSSchedulerApp> iter = new MultiListStartTimeIterator(
+        appsNowMaybeRunnable);
+    FSSchedulerApp prev = null;
+    int numNowRunnable = 0;
+    while (iter.hasNext()) {
+      FSSchedulerApp next = iter.next();
+      if (next == prev) {
+        continue;
+      }
+
+      if (canAppBeRunnable(next.getQueue(), next.getUser())) {
+        trackRunnableApp(next);
+        AppSchedulable appSched = next.getAppSchedulable();
+        next.getQueue().makeAppRunnable(appSched);
+        if (!usersNonRunnableApps.remove(next.getUser(), appSched)) {
+          throw new IllegalStateException("Waiting app " + next
+              + " expected to be in usersNonRunnableApps");
+        }
+
+        // No more than one app per list will be able to be made runnable, so
+        // we can stop looking after we've found that many
+        if (numNowRunnable >= appsNowMaybeRunnable.size()) {
+          break;
+        }
+      }
+
+      prev = next;
+    }
+  }
+  
+  /**
+   * Stops tracking the given non-runnable app
+   */
+  public void untrackNonRunnableApp(FSSchedulerApp app) {
+    usersNonRunnableApps.remove(app.getUser(), app.getAppSchedulable());
+  }
+
+  /**
+   * Traverses the queue hierarchy under the given queue to gather all lists
+   * of non-runnable applications.
+   */
+  private void gatherPossiblyRunnableAppLists(FSQueue queue,
+      List<List<AppSchedulable>> appLists) {
+    if (queue.getNumRunnableApps() < queueMgr.getQueueMaxApps(queue.getName())) {
+      if (queue instanceof FSLeafQueue) {
+        appLists.add(((FSLeafQueue)queue).getNonRunnableAppSchedulables());
+      } else {
+        for (FSQueue child : queue.getChildQueues()) {
+          gatherPossiblyRunnableAppLists(child, appLists);
+        }
+      }
+    }
+  }
+
+  /**
+   * Takes a list of lists, each of which is ordered by start time, and returns
+   * their elements in order of start time.
+   * 
+   * We maintain positions in each of the lists.  Each next() call advances
+   * the position in one of the lists.  We maintain a heap that orders lists
+   * by the start time of the app in the current position in that list.
+   * This allows us to pick which list to advance in O(log(num lists)) instead
+   * of O(num lists) time.
+   */
+  private static class MultiListStartTimeIterator implements
+      Iterator<FSSchedulerApp> {
+
+    private List<AppSchedulable>[] appLists;
+    private int[] curPositionsInAppLists;
+    private PriorityQueue<IndexAndTime> appListsByCurStartTime;
+
+    @SuppressWarnings("unchecked")
+    public MultiListStartTimeIterator(List<List<AppSchedulable>> appListList) {
+      appLists = appListList.toArray(new List[appListList.size()]);
+      curPositionsInAppLists = new int[appLists.length];
+      appListsByCurStartTime = new PriorityQueue<IndexAndTime>();
+      for (int i = 0; i < appLists.length; i++) {
+        long time = appLists[i].isEmpty() ? Long.MAX_VALUE : appLists[i].get(0)
+            .getStartTime();
+        appListsByCurStartTime.add(new IndexAndTime(i, time));
+      }
+    }
+
+    @Override
+    public boolean hasNext() {
+      return !appListsByCurStartTime.isEmpty()
+          && appListsByCurStartTime.peek().time != Long.MAX_VALUE;
+    }
+
+    @Override
+    public FSSchedulerApp next() {
+      IndexAndTime indexAndTime = appListsByCurStartTime.remove();
+      int nextListIndex = indexAndTime.index;
+      AppSchedulable next = appLists[nextListIndex]
+          .get(curPositionsInAppLists[nextListIndex]);
+      curPositionsInAppLists[nextListIndex]++;
+
+      if (curPositionsInAppLists[nextListIndex] < appLists[nextListIndex].size()) {
+        indexAndTime.time = appLists[nextListIndex]
+            .get(curPositionsInAppLists[nextListIndex]).getStartTime();
+      } else {
+        indexAndTime.time = Long.MAX_VALUE;
+      }
+      appListsByCurStartTime.add(indexAndTime);
+
+      return next.getApp();
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException("Remove not supported");
+    }
+
+    private static class IndexAndTime implements Comparable<IndexAndTime> {
+      public int index;
+      public long time;
+
+      public IndexAndTime(int index, long time) {
+        this.index = index;
+        this.time = time;
+      }
+
+      @Override
+      public int compareTo(IndexAndTime o) {
+        return time < o.time ? -1 : (time > o.time ? 1 : 0);
+      }
+
+      @Override
+      public boolean equals(Object o) {
+        if (!(o instanceof IndexAndTime)) {
+          return false;
+        }
+        IndexAndTime other = (IndexAndTime)o;
+        return other.time == time;
+      }
+
+      @Override
+      public int hashCode() {
+        return (int)time;
+      }
+    }
+  }
+}

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

@@ -89,7 +89,8 @@ public class QueueManager {
   private final Map<String, FSQueue> queues = new HashMap<String, FSQueue>();
   private FSParentQueue rootQueue;
 
-  private volatile QueueManagerInfo info = new QueueManagerInfo();
+  @VisibleForTesting
+  volatile QueueManagerInfo info = new QueueManagerInfo();
   @VisibleForTesting
   volatile QueuePlacementPolicy placementPolicy;
   

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerLeafQueueInfo.java

@@ -39,7 +39,7 @@ public class FairSchedulerLeafQueueInfo extends FairSchedulerQueueInfo {
   
   public FairSchedulerLeafQueueInfo(FSLeafQueue queue, FairScheduler scheduler) {
     super(queue, scheduler);
-    Collection<AppSchedulable> apps = queue.getAppSchedulables();
+    Collection<AppSchedulable> apps = queue.getRunnableAppSchedulables();
     for (AppSchedulable app : apps) {
       if (app.getApp().isPending()) {
         numPendingApps++;
@@ -47,6 +47,7 @@ public class FairSchedulerLeafQueueInfo extends FairSchedulerQueueInfo {
         numActiveApps++;
       }
     }
+    numPendingApps += queue.getNonRunnableAppSchedulables().size();
   }
   
   public int getNumActiveApplications() {

+ 8 - 7
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

@@ -596,23 +596,24 @@ public class TestCapacityScheduler {
     public void testConcurrentAccessOnApplications() throws Exception {
       CapacityScheduler cs = new CapacityScheduler();
       verifyConcurrentAccessOnApplications(
-          cs.applications, FiCaSchedulerApp.class);
+          cs.applications, FiCaSchedulerApp.class, Queue.class);
     }
 
-    public static <T extends SchedulerApplication>
+    public static <T extends SchedulerApplication, Q extends Queue>
         void verifyConcurrentAccessOnApplications(
-            final Map<ApplicationAttemptId, T> applications, Class<T> clazz)
+            final Map<ApplicationAttemptId, T> applications, Class<T> appClazz,
+            final Class<Q> queueClazz)
                 throws Exception {
       final int size = 10000;
       final ApplicationId appId = ApplicationId.newInstance(0, 0);
-      final Constructor<T> ctor = clazz.getDeclaredConstructor(
-          ApplicationAttemptId.class, String.class, Queue.class,
+      final Constructor<T> ctor = appClazz.getDeclaredConstructor(
+          ApplicationAttemptId.class, String.class, queueClazz,
           ActiveUsersManager.class, RMContext.class);
 
       ApplicationAttemptId appAttemptId0
           = ApplicationAttemptId.newInstance(appId, 0);
       applications.put(appAttemptId0, ctor.newInstance(
-              appAttemptId0, null, mock(Queue.class), null, null));
+              appAttemptId0, null, mock(queueClazz), null, null));
       assertNotNull(applications.get(appAttemptId0));
 
       // Imitating the thread of scheduler that will add and remove apps
@@ -627,7 +628,7 @@ public class TestCapacityScheduler {
                 = ApplicationAttemptId.newInstance(appId, i);
             try {
               applications.put(appAttemptId, ctor.newInstance(
-                  appAttemptId, null, mock(Queue.class), null, null));
+                  appAttemptId, null, mock(queueClazz), null, null));
             } catch (Exception e) {
               failed.set(true);
               finished.set(true);

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSSchedulerApp.java

@@ -24,7 +24,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.util.Clock;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -53,7 +52,7 @@ public class TestFSSchedulerApp {
 
   @Test
   public void testDelayScheduling() {
-    Queue queue = Mockito.mock(Queue.class);
+    FSLeafQueue queue = Mockito.mock(FSLeafQueue.class);
     Priority prio = Mockito.mock(Priority.class);
     Mockito.when(prio.getPriority()).thenReturn(1);
     double nodeLocalityThreshold = .5;
@@ -110,7 +109,7 @@ public class TestFSSchedulerApp {
   @Test
   public void testDelaySchedulingForContinuousScheduling()
           throws InterruptedException {
-    Queue queue = Mockito.mock(Queue.class);
+    FSLeafQueue queue = Mockito.mock(FSLeafQueue.class);
     Priority prio = Mockito.mock(Priority.class);
     Mockito.when(prio.getPriority()).thenReturn(1);
 
@@ -170,7 +169,7 @@ public class TestFSSchedulerApp {
    * no tin use), the least restrictive locality level is returned.
    */
   public void testLocalityLevelWithoutDelays() {
-    Queue queue = Mockito.mock(Queue.class);
+    FSLeafQueue queue = Mockito.mock(FSLeafQueue.class);
     Priority prio = Mockito.mock(Priority.class);
     Mockito.when(prio.getPriority()).thenReturn(1);
 

+ 167 - 16
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

@@ -100,7 +100,7 @@ import com.google.common.collect.Sets;
 
 public class TestFairScheduler {
 
-  private class MockClock implements Clock {
+  static class MockClock implements Clock {
     private long time = 0;
     @Override
     public long getTime() {
@@ -613,9 +613,9 @@ public class TestFairScheduler {
         appAttemptId, "default", "user1");
     scheduler.handle(appAddedEvent);
     assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true)
-        .getAppSchedulables().size());
+        .getRunnableAppSchedulables().size());
     assertEquals(0, scheduler.getQueueManager().getLeafQueue("default", true)
-        .getAppSchedulables().size());
+        .getRunnableAppSchedulables().size());
     assertEquals("root.user1", rmApp.getQueue());
 
     conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "false");
@@ -625,11 +625,11 @@ public class TestFairScheduler {
         createAppAttemptId(2, 1), "default", "user2");
     scheduler.handle(appAddedEvent2);
     assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true)
-        .getAppSchedulables().size());
+        .getRunnableAppSchedulables().size());
     assertEquals(1, scheduler.getQueueManager().getLeafQueue("default", true)
-        .getAppSchedulables().size());
+        .getRunnableAppSchedulables().size());
     assertEquals(0, scheduler.getQueueManager().getLeafQueue("user2", true)
-        .getAppSchedulables().size());
+        .getRunnableAppSchedulables().size());
   }
 
   @Test
@@ -821,7 +821,7 @@ public class TestFairScheduler {
 
     // That queue should have one app
     assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true)
-        .getAppSchedulables().size());
+        .getRunnableAppSchedulables().size());
 
     AppRemovedSchedulerEvent appRemovedEvent1 = new AppRemovedSchedulerEvent(
         createAppAttemptId(1, 1), RMAppAttemptState.FINISHED);
@@ -831,7 +831,7 @@ public class TestFairScheduler {
 
     // Queue should have no apps
     assertEquals(0, scheduler.getQueueManager().getLeafQueue("user1", true)
-        .getAppSchedulables().size());
+        .getRunnableAppSchedulables().size());
   }
 
   @Test
@@ -2400,7 +2400,158 @@ public class TestFairScheduler {
   public void testConcurrentAccessOnApplications() throws Exception {
     FairScheduler fs = new FairScheduler();
     TestCapacityScheduler.verifyConcurrentAccessOnApplications(
-        fs.applications, FSSchedulerApp.class);
+        fs.applications, FSSchedulerApp.class, FSLeafQueue.class);
+  }
+  
+  
+  private void verifyAppRunnable(ApplicationAttemptId attId, boolean runnable) {
+    FSSchedulerApp app = scheduler.applications.get(attId);
+    FSLeafQueue queue = app.getQueue();
+    Collection<AppSchedulable> runnableApps =
+        queue.getRunnableAppSchedulables();
+    Collection<AppSchedulable> nonRunnableApps =
+        queue.getNonRunnableAppSchedulables();
+    assertEquals(runnable, runnableApps.contains(app.getAppSchedulable()));
+    assertEquals(!runnable, nonRunnableApps.contains(app.getAppSchedulable()));
+  }
+  
+  private void verifyQueueNumRunnable(String queueName, int numRunnableInQueue,
+      int numNonRunnableInQueue) {
+    FSLeafQueue queue = scheduler.getQueueManager().getLeafQueue(queueName, false);
+    assertEquals(numRunnableInQueue,
+        queue.getRunnableAppSchedulables().size());
+    assertEquals(numNonRunnableInQueue,
+        queue.getNonRunnableAppSchedulables().size());
+  }
+  
+  @Test
+  public void testUserAndQueueMaxRunningApps() throws Exception {
+    Configuration conf = createConfiguration();
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"queue1\">");
+    out.println("<maxRunningApps>2</maxRunningApps>");
+    out.println("</queue>");
+    out.println("<user name=\"user1\">");
+    out.println("<maxRunningApps>1</maxRunningApps>");
+    out.println("</user>");
+    out.println("</allocations>");
+    out.close();
+    
+    QueueManager queueManager = scheduler.getQueueManager();
+    queueManager.initialize();
+    
+    // exceeds no limits
+    ApplicationAttemptId attId1 = createSchedulingRequest(1024, "queue1", "user1");
+    verifyAppRunnable(attId1, true);
+    verifyQueueNumRunnable("queue1", 1, 0);
+    // exceeds user limit
+    ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue2", "user1");
+    verifyAppRunnable(attId2, false);
+    verifyQueueNumRunnable("queue2", 0, 1);
+    // exceeds no limits
+    ApplicationAttemptId attId3 = createSchedulingRequest(1024, "queue1", "user2");
+    verifyAppRunnable(attId3, true);
+    verifyQueueNumRunnable("queue1", 2, 0);
+    // exceeds queue limit
+    ApplicationAttemptId attId4 = createSchedulingRequest(1024, "queue1", "user2");
+    verifyAppRunnable(attId4, false);
+    verifyQueueNumRunnable("queue1", 2, 1);
+    
+    // Remove app 1 and both app 2 and app 4 should becomes runnable in its place
+    AppRemovedSchedulerEvent appRemovedEvent1 = new AppRemovedSchedulerEvent(
+        attId1, RMAppAttemptState.FINISHED);
+    scheduler.handle(appRemovedEvent1);
+    verifyAppRunnable(attId2, true);
+    verifyQueueNumRunnable("queue2", 1, 0);
+    verifyAppRunnable(attId4, true);
+    verifyQueueNumRunnable("queue1", 2, 0);
+    
+    // A new app to queue1 should not be runnable
+    ApplicationAttemptId attId5 = createSchedulingRequest(1024, "queue1", "user2");
+    verifyAppRunnable(attId5, false);
+    verifyQueueNumRunnable("queue1", 2, 1);
+  }
+  
+  @Test
+  public void testMaxRunningAppsHierarchicalQueues() throws Exception {
+    Configuration conf = createConfiguration();
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+    MockClock clock = new MockClock();
+    scheduler.setClock(clock);
+
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"queue1\">");
+    out.println("  <maxRunningApps>3</maxRunningApps>");
+    out.println("  <queue name=\"sub1\"></queue>");
+    out.println("  <queue name=\"sub2\"></queue>");
+    out.println("  <queue name=\"sub3\">");
+    out.println("    <maxRunningApps>1</maxRunningApps>");
+    out.println("  </queue>");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+    
+    QueueManager queueManager = scheduler.getQueueManager();
+    queueManager.initialize();
+    
+    // exceeds no limits
+    ApplicationAttemptId attId1 = createSchedulingRequest(1024, "queue1.sub1", "user1");
+    verifyAppRunnable(attId1, true);
+    verifyQueueNumRunnable("queue1.sub1", 1, 0);
+    clock.tick(10);
+    // exceeds no limits
+    ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1.sub3", "user1");
+    verifyAppRunnable(attId2, true);
+    verifyQueueNumRunnable("queue1.sub3", 1, 0);
+    clock.tick(10);
+    // exceeds no limits
+    ApplicationAttemptId attId3 = createSchedulingRequest(1024, "queue1.sub2", "user1");
+    verifyAppRunnable(attId3, true);
+    verifyQueueNumRunnable("queue1.sub2", 1, 0);
+    clock.tick(10);
+    // exceeds queue1 limit
+    ApplicationAttemptId attId4 = createSchedulingRequest(1024, "queue1.sub2", "user1");
+    verifyAppRunnable(attId4, false);
+    verifyQueueNumRunnable("queue1.sub2", 1, 1);
+    clock.tick(10);
+    // exceeds sub3 limit
+    ApplicationAttemptId attId5 = createSchedulingRequest(1024, "queue1.sub3", "user1");
+    verifyAppRunnable(attId5, false);
+    verifyQueueNumRunnable("queue1.sub3", 1, 1);
+    clock.tick(10);
+    
+    // Even though the app was removed from sub3, the app from sub2 gets to go
+    // because it came in first
+    AppRemovedSchedulerEvent appRemovedEvent1 = new AppRemovedSchedulerEvent(
+        attId2, RMAppAttemptState.FINISHED);
+    scheduler.handle(appRemovedEvent1);
+    verifyAppRunnable(attId4, true);
+    verifyQueueNumRunnable("queue1.sub2", 2, 0);
+    verifyAppRunnable(attId5, false);
+    verifyQueueNumRunnable("queue1.sub3", 0, 1);
+
+    // Now test removal of a non-runnable app
+    AppRemovedSchedulerEvent appRemovedEvent2 = new AppRemovedSchedulerEvent(
+        attId5, RMAppAttemptState.KILLED);
+    scheduler.handle(appRemovedEvent2);
+    assertEquals(0, scheduler.maxRunningEnforcer.usersNonRunnableApps
+        .get("user1").size());
+    // verify app gone in queue accounting
+    verifyQueueNumRunnable("queue1.sub3", 0, 0);
+    // verify it doesn't become runnable when there would be space for it
+    AppRemovedSchedulerEvent appRemovedEvent3 = new AppRemovedSchedulerEvent(
+        attId4, RMAppAttemptState.FINISHED);
+    scheduler.handle(appRemovedEvent3);
+    verifyQueueNumRunnable("queue1.sub2", 1, 0);
+    verifyQueueNumRunnable("queue1.sub3", 0, 0);
   }
 
   @Test (timeout = 10000)
@@ -2499,23 +2650,23 @@ public class TestFairScheduler {
     
     // Should get put into jerry
     createSchedulingRequest(1024, "jerry", "someuser");
-    assertEquals(1, jerryQueue.getAppSchedulables().size());
+    assertEquals(1, jerryQueue.getRunnableAppSchedulables().size());
 
     // Should get forced into default
     createSchedulingRequest(1024, "newqueue", "someuser");
-    assertEquals(1, jerryQueue.getAppSchedulables().size());
-    assertEquals(1, defaultQueue.getAppSchedulables().size());
+    assertEquals(1, jerryQueue.getRunnableAppSchedulables().size());
+    assertEquals(1, defaultQueue.getRunnableAppSchedulables().size());
     
     // Would get put into someuser because of user-as-default-queue, but should
     // be forced into default
     createSchedulingRequest(1024, "default", "someuser");
-    assertEquals(1, jerryQueue.getAppSchedulables().size());
-    assertEquals(2, defaultQueue.getAppSchedulables().size());
+    assertEquals(1, jerryQueue.getRunnableAppSchedulables().size());
+    assertEquals(2, defaultQueue.getRunnableAppSchedulables().size());
     
     // Should get put into jerry because of user-as-default-queue
     createSchedulingRequest(1024, "default", "jerry");
-    assertEquals(2, jerryQueue.getAppSchedulables().size());
-    assertEquals(2, defaultQueue.getAppSchedulables().size());
+    assertEquals(2, jerryQueue.getRunnableAppSchedulables().size());
+    assertEquals(2, defaultQueue.getRunnableAppSchedulables().size());
   }
 
   @SuppressWarnings("resource")

+ 152 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestMaxRunningAppsEnforcer.java

@@ -0,0 +1,152 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestMaxRunningAppsEnforcer {
+  private QueueManager queueManager;
+  private Map<String, Integer> queueMaxApps;
+  private Map<String, Integer> userMaxApps;
+  private MaxRunningAppsEnforcer maxAppsEnforcer;
+  private int appNum;
+  private TestFairScheduler.MockClock clock;
+  
+  @Before
+  public void setup() throws Exception {
+    clock = new TestFairScheduler.MockClock();
+    FairScheduler scheduler = mock(FairScheduler.class);
+    when(scheduler.getConf()).thenReturn(
+        new FairSchedulerConfiguration(new Configuration()));
+    when(scheduler.getClock()).thenReturn(clock);
+    
+    queueManager = new QueueManager(scheduler);
+    queueManager.initialize();
+    
+    queueMaxApps = queueManager.info.queueMaxApps;
+    userMaxApps = queueManager.info.userMaxApps;
+    maxAppsEnforcer = new MaxRunningAppsEnforcer(queueManager);
+    appNum = 0;
+  }
+  
+  private FSSchedulerApp addApp(FSLeafQueue queue, String user) {
+    ApplicationId appId = ApplicationId.newInstance(0l, appNum++);
+    ApplicationAttemptId attId = ApplicationAttemptId.newInstance(appId, 0);
+    boolean runnable = maxAppsEnforcer.canAppBeRunnable(queue, user);
+    FSSchedulerApp app = new FSSchedulerApp(attId, user, queue, null, null);
+    queue.addApp(app, runnable);
+    if (runnable) {
+      maxAppsEnforcer.trackRunnableApp(app);
+    } else {
+      maxAppsEnforcer.trackNonRunnableApp(app);
+    }
+    return app;
+  }
+  
+  private void removeApp(FSSchedulerApp app) {
+    app.getQueue().removeApp(app);
+    maxAppsEnforcer.updateRunnabilityOnAppRemoval(app);
+  }
+  
+  @Test
+  public void testRemoveDoesNotEnableAnyApp() {
+    FSLeafQueue leaf1 = queueManager.getLeafQueue("root.queue1", true);
+    FSLeafQueue leaf2 = queueManager.getLeafQueue("root.queue2", true);
+    queueMaxApps.put("root", 2);
+    queueMaxApps.put("root.queue1", 1);
+    queueMaxApps.put("root.queue2", 1);
+    FSSchedulerApp app1 = addApp(leaf1, "user");
+    addApp(leaf2, "user");
+    addApp(leaf2, "user");
+    assertEquals(1, leaf1.getRunnableAppSchedulables().size());
+    assertEquals(1, leaf2.getRunnableAppSchedulables().size());
+    assertEquals(1, leaf2.getNonRunnableAppSchedulables().size());
+    removeApp(app1);
+    assertEquals(0, leaf1.getRunnableAppSchedulables().size());
+    assertEquals(1, leaf2.getRunnableAppSchedulables().size());
+    assertEquals(1, leaf2.getNonRunnableAppSchedulables().size());
+  }
+  
+  @Test
+  public void testRemoveEnablesAppOnCousinQueue() {
+    FSLeafQueue leaf1 = queueManager.getLeafQueue("root.queue1.subqueue1.leaf1", true);
+    FSLeafQueue leaf2 = queueManager.getLeafQueue("root.queue1.subqueue2.leaf2", true);
+    queueMaxApps.put("root.queue1", 2);
+    FSSchedulerApp app1 = addApp(leaf1, "user");
+    addApp(leaf2, "user");
+    addApp(leaf2, "user");
+    assertEquals(1, leaf1.getRunnableAppSchedulables().size());
+    assertEquals(1, leaf2.getRunnableAppSchedulables().size());
+    assertEquals(1, leaf2.getNonRunnableAppSchedulables().size());
+    removeApp(app1);
+    assertEquals(0, leaf1.getRunnableAppSchedulables().size());
+    assertEquals(2, leaf2.getRunnableAppSchedulables().size());
+    assertEquals(0, leaf2.getNonRunnableAppSchedulables().size());
+  }
+  
+  @Test
+  public void testRemoveEnablesOneByQueueOneByUser() {
+    FSLeafQueue leaf1 = queueManager.getLeafQueue("root.queue1.leaf1", true);
+    FSLeafQueue leaf2 = queueManager.getLeafQueue("root.queue1.leaf2", true);
+    queueMaxApps.put("root.queue1.leaf1", 2);
+    userMaxApps.put("user1", 1);
+    FSSchedulerApp app1 = addApp(leaf1, "user1");
+    addApp(leaf1, "user2");
+    addApp(leaf1, "user3");
+    addApp(leaf2, "user1");
+    assertEquals(2, leaf1.getRunnableAppSchedulables().size());
+    assertEquals(1, leaf1.getNonRunnableAppSchedulables().size());
+    assertEquals(1, leaf2.getNonRunnableAppSchedulables().size());
+    removeApp(app1);
+    assertEquals(2, leaf1.getRunnableAppSchedulables().size());
+    assertEquals(1, leaf2.getRunnableAppSchedulables().size());
+    assertEquals(0, leaf1.getNonRunnableAppSchedulables().size());
+    assertEquals(0, leaf2.getNonRunnableAppSchedulables().size());
+  }
+  
+  @Test
+  public void testRemoveEnablingOrderedByStartTime() {
+    FSLeafQueue leaf1 = queueManager.getLeafQueue("root.queue1.subqueue1.leaf1", true);
+    FSLeafQueue leaf2 = queueManager.getLeafQueue("root.queue1.subqueue2.leaf2", true);
+    queueMaxApps.put("root.queue1", 2);
+    FSSchedulerApp app1 = addApp(leaf1, "user");
+    addApp(leaf2, "user");
+    addApp(leaf2, "user");
+    clock.tick(20);
+    addApp(leaf1, "user");
+    assertEquals(1, leaf1.getRunnableAppSchedulables().size());
+    assertEquals(1, leaf2.getRunnableAppSchedulables().size());
+    assertEquals(1, leaf1.getNonRunnableAppSchedulables().size());
+    assertEquals(1, leaf2.getNonRunnableAppSchedulables().size());
+    removeApp(app1);
+    assertEquals(0, leaf1.getRunnableAppSchedulables().size());
+    assertEquals(2, leaf2.getRunnableAppSchedulables().size());
+    assertEquals(0, leaf2.getNonRunnableAppSchedulables().size());
+  }
+  
+}

+ 2 - 1
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

@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.Task;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
@@ -518,7 +519,7 @@ public class TestFifoScheduler {
   public void testConcurrentAccessOnApplications() throws Exception {
     FifoScheduler fs = new FifoScheduler();
     TestCapacityScheduler.verifyConcurrentAccessOnApplications(
-        fs.applications, FiCaSchedulerApp.class);
+        fs.applications, FiCaSchedulerApp.class, Queue.class);
   }
 
   @SuppressWarnings("resource")