|
@@ -24,7 +24,9 @@ import java.security.AccessControlException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
import java.util.EnumSet;
|
|
|
+import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
@@ -67,6 +69,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
|
@@ -431,12 +434,51 @@ public class ClientRMService extends AbstractService implements
|
|
|
LongRange start = request.getStartRange();
|
|
|
LongRange finish = request.getFinishRange();
|
|
|
|
|
|
- List<ApplicationReport> reports = new ArrayList<ApplicationReport>();
|
|
|
- long count = 0;
|
|
|
- for (RMApp application : this.rmContext.getRMApps().values()) {
|
|
|
- if (++count > limit) {
|
|
|
- break;
|
|
|
+ final Map<ApplicationId, RMApp> apps = rmContext.getRMApps();
|
|
|
+ Iterator<RMApp> appsIter;
|
|
|
+ // If the query filters by queues, we can avoid considering apps outside
|
|
|
+ // of those queues by asking the scheduler for the apps in those queues.
|
|
|
+ if (queues != null && !queues.isEmpty()) {
|
|
|
+ // Construct an iterator over apps in given queues
|
|
|
+ // Collect list of lists to avoid copying all apps
|
|
|
+ final List<List<ApplicationAttemptId>> queueAppLists =
|
|
|
+ new ArrayList<List<ApplicationAttemptId>>();
|
|
|
+ for (String queue : queues) {
|
|
|
+ List<ApplicationAttemptId> appsInQueue = scheduler.getAppsInQueue(queue);
|
|
|
+ if (appsInQueue != null && !appsInQueue.isEmpty()) {
|
|
|
+ queueAppLists.add(appsInQueue);
|
|
|
+ }
|
|
|
}
|
|
|
+ appsIter = new Iterator<RMApp>() {
|
|
|
+ Iterator<List<ApplicationAttemptId>> appListIter = queueAppLists.iterator();
|
|
|
+ Iterator<ApplicationAttemptId> schedAppsIter;
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean hasNext() {
|
|
|
+ // Because queueAppLists has no empty lists, hasNext is whether the
|
|
|
+ // current list hasNext or whether there are any remaining lists
|
|
|
+ return (schedAppsIter != null && schedAppsIter.hasNext())
|
|
|
+ || appListIter.hasNext();
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public RMApp next() {
|
|
|
+ if (schedAppsIter == null || !schedAppsIter.hasNext()) {
|
|
|
+ schedAppsIter = appListIter.next().iterator();
|
|
|
+ }
|
|
|
+ return apps.get(schedAppsIter.next().getApplicationId());
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public void remove() {
|
|
|
+ throw new UnsupportedOperationException("Remove not supported");
|
|
|
+ }
|
|
|
+ };
|
|
|
+ } else {
|
|
|
+ appsIter = apps.values().iterator();
|
|
|
+ }
|
|
|
+
|
|
|
+ List<ApplicationReport> reports = new ArrayList<ApplicationReport>();
|
|
|
+ while (appsIter.hasNext() && reports.size() < limit) {
|
|
|
+ RMApp application = appsIter.next();
|
|
|
if (applicationTypes != null && !applicationTypes.isEmpty()) {
|
|
|
String appTypeToMatch = caseSensitive
|
|
|
? application.getApplicationType()
|
|
@@ -458,11 +500,6 @@ public class ClientRMService extends AbstractService implements
|
|
|
continue;
|
|
|
}
|
|
|
|
|
|
- if (queues != null && !queues.isEmpty() &&
|
|
|
- !queues.contains(application.getQueue())) {
|
|
|
- continue;
|
|
|
- }
|
|
|
-
|
|
|
if (start != null && !start.containsLong(application.getStartTime())) {
|
|
|
continue;
|
|
|
}
|
|
@@ -515,13 +552,12 @@ public class ClientRMService extends AbstractService implements
|
|
|
request.getRecursive());
|
|
|
List<ApplicationReport> appReports = EMPTY_APPS_REPORT;
|
|
|
if (request.getIncludeApplications()) {
|
|
|
- Collection<RMApp> apps = this.rmContext.getRMApps().values();
|
|
|
- appReports = new ArrayList<ApplicationReport>(
|
|
|
- apps.size());
|
|
|
- for (RMApp app : apps) {
|
|
|
- if (app.getQueue().equals(queueInfo.getQueueName())) {
|
|
|
- appReports.add(app.createAndGetApplicationReport(null, true));
|
|
|
- }
|
|
|
+ List<ApplicationAttemptId> apps =
|
|
|
+ scheduler.getAppsInQueue(request.getQueueName());
|
|
|
+ appReports = new ArrayList<ApplicationReport>(apps.size());
|
|
|
+ for (ApplicationAttemptId app : apps) {
|
|
|
+ RMApp rmApp = rmContext.getRMApps().get(app.getApplicationId());
|
|
|
+ appReports.add(rmApp.createAndGetApplicationReport(null, true));
|
|
|
}
|
|
|
}
|
|
|
queueInfo.setApplications(appReports);
|