You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ey...@apache.org on 2018/07/19 16:31:52 UTC

hadoop git commit: YARN-8501. Reduce complexity of RMWebServices getApps method. Contributed by Szilard Nemeth

Repository: hadoop
Updated Branches:
  refs/heads/trunk ccf2db7fc -> 5836e0a46


YARN-8501. Reduce complexity of RMWebServices getApps method.
           Contributed by Szilard Nemeth


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

Branch: refs/heads/trunk
Commit: 5836e0a46bf9793e0a61bb8ec46536f4a67d38d7
Parents: ccf2db7
Author: Eric Yang <ey...@apache.org>
Authored: Thu Jul 19 12:30:38 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Thu Jul 19 12:30:38 2018 -0400

----------------------------------------------------------------------
 .../hadoop/yarn/server/webapp/WebServices.java  |   2 +-
 .../webapp/ApplicationsRequestBuilder.java      | 231 ++++++++
 .../resourcemanager/webapp/RMWebServices.java   | 145 +----
 .../webapp/TestApplicationsRequestBuilder.java  | 529 +++++++++++++++++++
 4 files changed, 777 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5836e0a4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
index 03b1055..5bb5448 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
@@ -392,7 +392,7 @@ public class WebServices {
     response.setContentType(null);
   }
 
-  protected static Set<String>
+  public static Set<String>
       parseQueries(Set<String> queries, boolean isState) {
     Set<String> params = new HashSet<String>();
     if (!queries.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5836e0a4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ApplicationsRequestBuilder.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/webapp/ApplicationsRequestBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ApplicationsRequestBuilder.java
new file mode 100644
index 0000000..876d044
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ApplicationsRequestBuilder.java
@@ -0,0 +1,231 @@
+/*
+ * 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.webapp;
+
+import com.google.common.collect.Sets;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+        .CapacityScheduler;
+import org.apache.hadoop.yarn.webapp.BadRequestException;
+
+import java.io.IOException;
+import java.util.Set;
+
+import static org.apache.hadoop.yarn.server.webapp.WebServices.parseQueries;
+
+public class ApplicationsRequestBuilder {
+
+  private Set<String> statesQuery = Sets.newHashSet();
+  private Set<String> users = Sets.newHashSetWithExpectedSize(1);
+  private Set<String> queues = Sets.newHashSetWithExpectedSize(1);
+  private String limit = null;
+  private Long limitNumber;
+
+  // set values suitable in case both of begin/end not specified
+  private long startedTimeBegin = 0;
+  private long startedTimeEnd = Long.MAX_VALUE;
+  private long finishTimeBegin = 0;
+  private long finishTimeEnd = Long.MAX_VALUE;
+  private Set<String> appTypes = Sets.newHashSet();
+  private Set<String> appTags = Sets.newHashSet();
+  private ResourceManager rm;
+
+  private ApplicationsRequestBuilder() {
+  }
+
+  public static ApplicationsRequestBuilder create() {
+    return new ApplicationsRequestBuilder();
+  }
+
+  public ApplicationsRequestBuilder withStateQuery(String stateQuery) {
+    // stateQuery is deprecated.
+    if (stateQuery != null && !stateQuery.isEmpty()) {
+      statesQuery.add(stateQuery);
+    }
+    return this;
+  }
+
+  public ApplicationsRequestBuilder withStatesQuery(
+      Set<String> statesQuery) {
+    if (statesQuery != null) {
+      this.statesQuery.addAll(statesQuery);
+    }
+    return this;
+  }
+
+  public ApplicationsRequestBuilder withUserQuery(String userQuery) {
+    if (userQuery != null && !userQuery.isEmpty()) {
+      users.add(userQuery);
+    }
+    return this;
+  }
+
+  public ApplicationsRequestBuilder withQueueQuery(ResourceManager rm,
+      String queueQuery) {
+    this.rm = rm;
+    if (queueQuery != null && !queueQuery.isEmpty()) {
+      queues.add(queueQuery);
+    }
+    return this;
+  }
+
+  public ApplicationsRequestBuilder withLimit(String limit) {
+    if (limit != null && !limit.isEmpty()) {
+      this.limit = limit;
+    }
+    return this;
+  }
+
+  public ApplicationsRequestBuilder withStartedTimeBegin(
+      String startedBegin) {
+    if (startedBegin != null && !startedBegin.isEmpty()) {
+      startedTimeBegin = parseLongValue(startedBegin, "startedTimeBegin");
+    }
+    return this;
+  }
+
+  public ApplicationsRequestBuilder withStartedTimeEnd(String startedEnd) {
+    if (startedEnd != null && !startedEnd.isEmpty()) {
+      startedTimeEnd = parseLongValue(startedEnd, "startedTimeEnd");
+    }
+    return this;
+  }
+
+  public ApplicationsRequestBuilder withFinishTimeBegin(String finishBegin) {
+    if (finishBegin != null && !finishBegin.isEmpty()) {
+      finishTimeBegin = parseLongValue(finishBegin, "finishedTimeBegin");
+    }
+    return this;
+  }
+
+  public ApplicationsRequestBuilder withFinishTimeEnd(String finishEnd) {
+    if (finishEnd != null && !finishEnd.isEmpty()) {
+      finishTimeEnd = parseLongValue(finishEnd, "finishedTimeEnd");
+    }
+    return this;
+  }
+
+  public ApplicationsRequestBuilder withApplicationTypes(
+      Set<String> applicationTypes) {
+    if (applicationTypes !=  null) {
+      appTypes = parseQueries(applicationTypes, false);
+    }
+    return this;
+  }
+
+  public ApplicationsRequestBuilder withApplicationTags(
+      Set<String> applicationTags) {
+    if (applicationTags != null) {
+      appTags = parseQueries(applicationTags, false);
+    }
+    return this;
+  }
+
+  private void validate() {
+    queues.forEach(q -> validateQueueExists(rm, q));
+    validateLimit();
+    validateStartTime();
+    validateFinishTime();
+  }
+
+  private void validateQueueExists(ResourceManager rm, String queueQuery) {
+    ResourceScheduler rs = rm.getResourceScheduler();
+    if (rs instanceof CapacityScheduler) {
+      CapacityScheduler cs = (CapacityScheduler) rs;
+      try {
+        cs.getQueueInfo(queueQuery, false, false);
+      } catch (IOException e) {
+        throw new BadRequestException(e.getMessage());
+      }
+    }
+  }
+
+  private void validateLimit() {
+    if (limit != null) {
+      limitNumber = parseLongValue(limit, "limit");
+      if (limitNumber <= 0) {
+        throw new BadRequestException("limit value must be greater then 0");
+      }
+    }
+  }
+
+  private long parseLongValue(String strValue, String queryName) {
+    try {
+      return Long.parseLong(strValue);
+    } catch (NumberFormatException e) {
+      throw new BadRequestException(queryName + " value must be a number!");
+    }
+  }
+
+  private void validateStartTime() {
+    if (startedTimeBegin < 0) {
+      throw new BadRequestException("startedTimeBegin must be greater than 0");
+    }
+    if (startedTimeEnd < 0) {
+      throw new BadRequestException("startedTimeEnd must be greater than 0");
+    }
+    if (startedTimeBegin > startedTimeEnd) {
+      throw new BadRequestException(
+          "startedTimeEnd must be greater than startTimeBegin");
+    }
+  }
+
+  private void validateFinishTime() {
+    if (finishTimeBegin < 0) {
+      throw new BadRequestException("finishTimeBegin must be greater than 0");
+    }
+    if (finishTimeEnd < 0) {
+      throw new BadRequestException("finishTimeEnd must be greater than 0");
+    }
+    if (finishTimeBegin > finishTimeEnd) {
+      throw new BadRequestException(
+          "finishTimeEnd must be greater than finishTimeBegin");
+    }
+  }
+
+  public GetApplicationsRequest build() {
+    validate();
+    GetApplicationsRequest request = GetApplicationsRequest.newInstance();
+
+    Set<String> appStates = parseQueries(statesQuery, true);
+    if (!appStates.isEmpty()) {
+      request.setApplicationStates(appStates);
+    }
+    if (!users.isEmpty()) {
+      request.setUsers(users);
+    }
+    if (!queues.isEmpty()) {
+      request.setQueues(queues);
+    }
+    if (limitNumber != null) {
+      request.setLimit(limitNumber);
+    }
+    request.setStartRange(startedTimeBegin, startedTimeEnd);
+    request.setFinishRange(finishTimeBegin, finishTimeEnd);
+
+    if (!appTypes.isEmpty()) {
+      request.setApplicationTypes(appTypes);
+    }
+    if (!appTags.isEmpty()) {
+      request.setApplicationTags(appTags);
+    }
+
+    return request;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5836e0a4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.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/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 15b58d7..4527a02 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -482,7 +482,7 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
       @QueryParam(RMWSConsts.FINAL_STATUS) String finalStatusQuery,
       @QueryParam(RMWSConsts.USER) String userQuery,
       @QueryParam(RMWSConsts.QUEUE) String queueQuery,
-      @QueryParam(RMWSConsts.LIMIT) String count,
+      @QueryParam(RMWSConsts.LIMIT) String limit,
       @QueryParam(RMWSConsts.STARTED_TIME_BEGIN) String startedBegin,
       @QueryParam(RMWSConsts.STARTED_TIME_END) String startedEnd,
       @QueryParam(RMWSConsts.FINISHED_TIME_BEGIN) String finishBegin,
@@ -493,135 +493,22 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
 
     initForReadableEndpoints();
 
-    boolean checkCount = false;
-    boolean checkStart = false;
-    boolean checkEnd = false;
-    boolean checkAppTypes = false;
-    boolean checkAppStates = false;
-    boolean checkAppTags = false;
-    long countNum = 0;
-
-    // set values suitable in case both of begin/end not specified
-    long sBegin = 0;
-    long sEnd = Long.MAX_VALUE;
-    long fBegin = 0;
-    long fEnd = Long.MAX_VALUE;
-
-    if (count != null && !count.isEmpty()) {
-      checkCount = true;
-      countNum = Long.parseLong(count);
-      if (countNum <= 0) {
-        throw new BadRequestException("limit value must be greater then 0");
-      }
-    }
-
-    if (startedBegin != null && !startedBegin.isEmpty()) {
-      checkStart = true;
-      sBegin = Long.parseLong(startedBegin);
-      if (sBegin < 0) {
-        throw new BadRequestException(
-            "startedTimeBegin must be greater than 0");
-      }
-    }
-    if (startedEnd != null && !startedEnd.isEmpty()) {
-      checkStart = true;
-      sEnd = Long.parseLong(startedEnd);
-      if (sEnd < 0) {
-        throw new BadRequestException("startedTimeEnd must be greater than 0");
-      }
-    }
-    if (sBegin > sEnd) {
-      throw new BadRequestException(
-          "startedTimeEnd must be greater than startTimeBegin");
-    }
-
-    if (finishBegin != null && !finishBegin.isEmpty()) {
-      checkEnd = true;
-      fBegin = Long.parseLong(finishBegin);
-      if (fBegin < 0) {
-        throw new BadRequestException("finishTimeBegin must be greater than 0");
-      }
-    }
-    if (finishEnd != null && !finishEnd.isEmpty()) {
-      checkEnd = true;
-      fEnd = Long.parseLong(finishEnd);
-      if (fEnd < 0) {
-        throw new BadRequestException("finishTimeEnd must be greater than 0");
-      }
-    }
-    if (fBegin > fEnd) {
-      throw new BadRequestException(
-          "finishTimeEnd must be greater than finishTimeBegin");
-    }
-
-    Set<String> appTypes = parseQueries(applicationTypes, false);
-    if (!appTypes.isEmpty()) {
-      checkAppTypes = true;
-    }
-
-    Set<String> appTags = parseQueries(applicationTags, false);
-    if (!appTags.isEmpty()) {
-      checkAppTags = true;
-    }
-
-    // stateQuery is deprecated.
-    if (stateQuery != null && !stateQuery.isEmpty()) {
-      statesQuery.add(stateQuery);
-    }
-    Set<String> appStates = parseQueries(statesQuery, true);
-    if (!appStates.isEmpty()) {
-      checkAppStates = true;
-    }
-
-    GetApplicationsRequest request = GetApplicationsRequest.newInstance();
-
-    if (checkStart) {
-      request.setStartRange(sBegin, sEnd);
-    }
-
-    if (checkEnd) {
-      request.setFinishRange(fBegin, fEnd);
-    }
-
-    if (checkCount) {
-      request.setLimit(countNum);
-    }
-
-    if (checkAppTypes) {
-      request.setApplicationTypes(appTypes);
-    }
-
-    if (checkAppTags) {
-      request.setApplicationTags(appTags);
-    }
-
-    if (checkAppStates) {
-      request.setApplicationStates(appStates);
-    }
-
-    if (queueQuery != null && !queueQuery.isEmpty()) {
-      ResourceScheduler rs = rm.getResourceScheduler();
-      if (rs instanceof CapacityScheduler) {
-        CapacityScheduler cs = (CapacityScheduler) rs;
-        // validate queue exists
-        try {
-          cs.getQueueInfo(queueQuery, false, false);
-        } catch (IOException e) {
-          throw new BadRequestException(e.getMessage());
-        }
-      }
-      Set<String> queues = new HashSet<String>(1);
-      queues.add(queueQuery);
-      request.setQueues(queues);
-    }
-
-    if (userQuery != null && !userQuery.isEmpty()) {
-      Set<String> users = new HashSet<String>(1);
-      users.add(userQuery);
-      request.setUsers(users);
-    }
+    GetApplicationsRequest request =
+            ApplicationsRequestBuilder.create()
+                    .withStateQuery(stateQuery)
+                    .withStatesQuery(statesQuery)
+                    .withUserQuery(userQuery)
+                    .withQueueQuery(rm, queueQuery)
+                    .withLimit(limit)
+                    .withStartedTimeBegin(startedBegin)
+                    .withStartedTimeEnd(startedEnd)
+                    .withFinishTimeBegin(finishBegin)
+                    .withFinishTimeEnd(finishEnd)
+                    .withApplicationTypes(applicationTypes)
+                    .withApplicationTags(applicationTags)
+            .build();
 
-    List<ApplicationReport> appReports = null;
+    List<ApplicationReport> appReports;
     try {
       appReports = rm.getClientRMService().getApplications(request)
           .getApplicationList();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5836e0a4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestApplicationsRequestBuilder.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/webapp/TestApplicationsRequestBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestApplicationsRequestBuilder.java
new file mode 100644
index 0000000..7c9b711
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestApplicationsRequestBuilder.java
@@ -0,0 +1,529 @@
+/*
+ * 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.webapp;
+
+import com.google.common.collect.Sets;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.webapp.BadRequestException;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Set;
+
+import static org.apache.hadoop.yarn.server.webapp.WebServices.parseQueries;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestApplicationsRequestBuilder {
+
+  private GetApplicationsRequest getDefaultRequest() {
+    GetApplicationsRequest req = GetApplicationsRequest.newInstance();
+    req.setStartRange(0, Long.MAX_VALUE);
+    req.setFinishRange(0, Long.MAX_VALUE);
+    return req;
+  }
+
+  @Test
+  public void testDefaultRequest() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithNullStateQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withStateQuery(null).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithEmptyStateQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withStateQuery("").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidStateQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withStateQuery("invalidState").build();
+  }
+
+  @Test
+  public void testRequestWithValidStateQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withStateQuery(YarnApplicationState.NEW_SAVING.toString()).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    Set<String> appStates =
+        Sets.newHashSet(YarnApplicationState.NEW_SAVING.toString());
+    Set<String> appStatesLowerCase = parseQueries(appStates, true);
+    expectedRequest.setApplicationStates(appStatesLowerCase);
+
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithEmptyStateQueries() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withStatesQuery(Sets.newHashSet()).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidStateQueries() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withStatesQuery(Sets.newHashSet("a1", "a2", "")).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithNullStateQueries() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withStatesQuery(null).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithValidStateQueries() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withStatesQuery(
+            Sets.newHashSet(YarnApplicationState.NEW_SAVING.toString(),
+                YarnApplicationState.NEW.toString()))
+        .build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    Set<String> appStates =
+        Sets.newHashSet(YarnApplicationState.NEW_SAVING.toString(),
+            YarnApplicationState.NEW.toString());
+    Set<String> appStatesLowerCase = parseQueries(appStates, true);
+    expectedRequest.setApplicationStates(appStatesLowerCase);
+
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithNullUserQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withUserQuery(null).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithEmptyUserQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withUserQuery("").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithUserQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withUserQuery("user1").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setUsers(Sets.newHashSet("user1"));
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithNullQueueQuery() {
+    ResourceManager rm = mock(ResourceManager.class);
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withQueueQuery(rm, null).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithEmptyQueueQuery() {
+    ResourceManager rm = mock(ResourceManager.class);
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withQueueQuery(rm, "").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithQueueQueryExistingQueue() {
+    ResourceManager rm = mock(ResourceManager.class);
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withQueueQuery(rm, "queue1").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setQueues(Sets.newHashSet("queue1"));
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithQueueQueryNotExistingQueue() throws IOException {
+    CapacityScheduler cs = mock(CapacityScheduler.class);
+    when(cs.getQueueInfo(eq("queue1"), anyBoolean(), anyBoolean()))
+        .thenThrow(new IOException());
+    ResourceManager rm = mock(ResourceManager.class);
+    when(rm.getResourceScheduler()).thenReturn(cs);
+
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withQueueQuery(rm, "queue1").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setQueues(Sets.newHashSet("queue1"));
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithNullLimitQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withLimit(null).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithEmptyLimitQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withLimit("").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidLimitQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withLimit("bla").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidNegativeLimitQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withLimit("-10").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithValidLimitQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withLimit("999").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setLimit(999L);
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithNullStartedTimeBeginQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withStartedTimeBegin(null).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithEmptyStartedTimeBeginQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withStartedTimeBegin("").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidStartedTimeBeginQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withStartedTimeBegin("bla").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidNegativeStartedTimeBeginQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withStartedTimeBegin("-1").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithValidStartedTimeBeginQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withStartedTimeBegin("999").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setStartRange(999L, Long.MAX_VALUE);
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithNullStartedTimeEndQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withStartedTimeEnd(null).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithEmptywithStartedTimeEndQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withStartedTimeEnd("").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidStartedTimeEndQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withStartedTimeEnd("bla").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidNegativeStartedTimeEndQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withStartedTimeEnd("-1").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithValidStartedTimeEndQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withStartedTimeEnd("999").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setStartRange(0L, 999L);
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithNullFinishedTimeBeginQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withFinishTimeBegin(null).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithEmptyFinishedTimeBeginQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withFinishTimeBegin("").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidFinishedTimeBeginQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withFinishTimeBegin("bla").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidNegativeFinishedTimeBeginQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withFinishTimeBegin("-1").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithValidFinishedTimeBeginQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withFinishTimeBegin("999").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setFinishRange(999L, Long.MAX_VALUE);
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithNullFinishedTimeEndQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withFinishTimeEnd(null).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithEmptyFinishTimeEndQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withFinishTimeEnd("").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidFinishTimeEndQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withFinishTimeEnd("bla").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidNegativeFinishedTimeEndQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withFinishTimeEnd("-1").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithValidFinishTimeEndQuery() {
+    GetApplicationsRequest request =
+        ApplicationsRequestBuilder.create().withFinishTimeEnd("999").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setFinishRange(0L, 999L);
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithValidStartTimeRangeQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withStartedTimeBegin("1000").withStartedTimeEnd("2000").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setStartRange(1000L, 2000L);
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidStartTimeRangeQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withStartedTimeBegin("2000").withStartedTimeEnd("1000").build();
+  }
+
+  @Test
+  public void testRequestWithValidFinishTimeRangeQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withFinishTimeBegin("1000").withFinishTimeEnd("2000").build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setFinishRange(1000L, 2000L);
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test(expected = BadRequestException.class)
+  public void testRequestWithInvalidFinishTimeRangeQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withFinishTimeBegin("2000").withFinishTimeEnd("1000").build();
+  }
+
+  @Test
+  public void testRequestWithNullApplicationTypesQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withApplicationTypes(null).build();
+  }
+
+  @Test
+  public void testRequestWithEmptyApplicationTypesQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withApplicationTypes(Sets.newHashSet()).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setApplicationTypes(Sets.newHashSet());
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithValidApplicationTypesQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withApplicationTypes(Sets.newHashSet("type1")).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setApplicationTypes(Sets.newHashSet("type1"));
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithNullApplicationTagsQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withApplicationTags(null).build();
+  }
+
+  @Test
+  public void testRequestWithEmptyApplicationTagsQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withApplicationTags(Sets.newHashSet()).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setApplicationTags(Sets.newHashSet());
+    assertEquals(expectedRequest, request);
+  }
+
+  @Test
+  public void testRequestWithValidApplicationTagsQuery() {
+    GetApplicationsRequest request = ApplicationsRequestBuilder.create()
+        .withApplicationTags(Sets.newHashSet("tag1")).build();
+
+    GetApplicationsRequest expectedRequest = getDefaultRequest();
+    expectedRequest.setApplicationTags(Sets.newHashSet("tag1"));
+    assertEquals(expectedRequest, request);
+  }
+}
\ No newline at end of file


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