瀏覽代碼

YARN-9214. Add AbstractYarnScheduler#getValidQueues method to remove duplication. Contributed by Wanqiang Ji.

Yufei Gu 6 年之前
父節點
當前提交
2f752830ba

+ 27 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java

@@ -771,16 +771,9 @@ public abstract class AbstractYarnScheduler
         LOG.warn(e.toString());
         throw new YarnException(e);
       }
-      // check if source queue is a valid
-      List<ApplicationAttemptId> apps = getAppsInQueue(sourceQueue);
-      if (apps == null) {
-        String errMsg =
-            "The specified Queue: " + sourceQueue + " doesn't exist";
-        LOG.warn(errMsg);
-        throw new YarnException(errMsg);
-      }
+
       // generate move events for each pending/running app
-      for (ApplicationAttemptId appAttemptId : apps) {
+      for (ApplicationAttemptId appAttemptId : getAppsFromQueue(sourceQueue)) {
         this.rmContext.getDispatcher().getEventHandler()
             .handle(new RMAppManagerEvent(appAttemptId.getApplicationId(),
                 destQueue, RMAppManagerEventType.APP_MOVE));
@@ -795,15 +788,8 @@ public abstract class AbstractYarnScheduler
       throws YarnException {
     writeLock.lock();
     try {
-      // check if queue is a valid
-      List<ApplicationAttemptId> apps = getAppsInQueue(queueName);
-      if (apps == null) {
-        String errMsg = "The specified Queue: " + queueName + " doesn't exist";
-        LOG.warn(errMsg);
-        throw new YarnException(errMsg);
-      }
       // generate kill events for each pending/running app
-      for (ApplicationAttemptId app : apps) {
+      for (ApplicationAttemptId app : getAppsFromQueue(queueName)) {
         this.rmContext.getDispatcher().getEventHandler().handle(
             new RMAppEvent(app.getApplicationId(), RMAppEventType.KILL,
                 "Application killed due to expiry of reservation queue "
@@ -1529,4 +1515,28 @@ public abstract class AbstractYarnScheduler
   public void resetSchedulerMetrics() {
     // reset scheduler metrics
   }
+
+  /**
+   * Gets the apps from a given queue.
+   *
+   * Mechanics:
+   * 1. Get all {@link ApplicationAttemptId}s in the given queue by
+   * {@link #getAppsInQueue(String)} method.
+   * 2. Always need to check validity for the given queue by the returned
+   * values.
+   *
+   * @param queueName queue name
+   * @return a collection of app attempt ids in the given queue, it maybe empty.
+   * @throws YarnException if {@link #getAppsInQueue(String)} return null, will
+   * throw this exception.
+   */
+  private List<ApplicationAttemptId> getAppsFromQueue(String queueName)
+      throws YarnException {
+    List<ApplicationAttemptId> apps = getAppsInQueue(queueName);
+    if (apps == null) {
+      throw new YarnException("The specified queue: " + queueName
+          + " doesn't exist");
+    }
+    return apps;
+  }
 }