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 cu...@apache.org on 2017/09/22 00:59:05 UTC

[44/50] [abbrv] hadoop git commit: YARN-5412. Create a proxy chain for ResourceManager REST API in the Router. (Contributed by Giovanni Matteo Fumarola via curino)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/PassThroughRESTRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/PassThroughRESTRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/PassThroughRESTRequestInterceptor.java
new file mode 100644
index 0000000..ea985a2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/PassThroughRESTRequestInterceptor.java
@@ -0,0 +1,339 @@
+/**
+ * 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.router.webapp;
+
+import java.io.IOException;
+import java.util.Set;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.Response;
+
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.DelegationToken;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
+
+/**
+ * Mock intercepter that does not do anything other than forwarding it to the
+ * next intercepter in the chain.
+ */
+public class PassThroughRESTRequestInterceptor
+    extends AbstractRESTRequestInterceptor {
+
+  @Override
+  public AppAttemptsInfo getAppAttempts(HttpServletRequest hsr, String appId) {
+    return getNextInterceptor().getAppAttempts(hsr, appId);
+  }
+
+  @Override
+  public AppAttemptInfo getAppAttempt(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId) {
+    return getNextInterceptor().getAppAttempt(req, res, appId, appAttemptId);
+  }
+
+  @Override
+  public ContainersInfo getContainers(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId) {
+    return getNextInterceptor().getContainers(req, res, appId, appAttemptId);
+  }
+
+  @Override
+  public ContainerInfo getContainer(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId,
+      String containerId) {
+    return getNextInterceptor().getContainer(req, res, appId, appAttemptId,
+        containerId);
+  }
+
+  @Override
+  public ClusterInfo get() {
+    return getNextInterceptor().get();
+  }
+
+  @Override
+  public ClusterInfo getClusterInfo() {
+    return getNextInterceptor().getClusterInfo();
+  }
+
+  @Override
+  public ClusterMetricsInfo getClusterMetricsInfo() {
+    return getNextInterceptor().getClusterMetricsInfo();
+  }
+
+  @Override
+  public SchedulerTypeInfo getSchedulerInfo() {
+    return getNextInterceptor().getSchedulerInfo();
+  }
+
+  @Override
+  public String dumpSchedulerLogs(String time, HttpServletRequest hsr)
+      throws IOException {
+    return getNextInterceptor().dumpSchedulerLogs(time, hsr);
+  }
+
+  @Override
+  public NodesInfo getNodes(String states) {
+    return getNextInterceptor().getNodes(states);
+  }
+
+  @Override
+  public NodeInfo getNode(String nodeId) {
+    return getNextInterceptor().getNode(nodeId);
+  }
+
+  @Override
+  public AppsInfo getApps(HttpServletRequest hsr, String stateQuery,
+      Set<String> statesQuery, String finalStatusQuery, String userQuery,
+      String queueQuery, String count, String startedBegin, String startedEnd,
+      String finishBegin, String finishEnd, Set<String> applicationTypes,
+      Set<String> applicationTags, Set<String> unselectedFields) {
+    return getNextInterceptor().getApps(hsr, stateQuery, statesQuery,
+        finalStatusQuery, userQuery, queueQuery, count, startedBegin,
+        startedEnd, finishBegin, finishEnd, applicationTypes, applicationTags,
+        unselectedFields);
+  }
+
+  @Override
+  public ActivitiesInfo getActivities(HttpServletRequest hsr, String nodeId) {
+    return getNextInterceptor().getActivities(hsr, nodeId);
+  }
+
+  @Override
+  public AppActivitiesInfo getAppActivities(HttpServletRequest hsr,
+      String appId, String time) {
+    return getNextInterceptor().getAppActivities(hsr, appId, time);
+  }
+
+  @Override
+  public ApplicationStatisticsInfo getAppStatistics(HttpServletRequest hsr,
+      Set<String> stateQueries, Set<String> typeQueries) {
+    return getNextInterceptor().getAppStatistics(hsr, stateQueries,
+        typeQueries);
+  }
+
+  @Override
+  public AppInfo getApp(HttpServletRequest hsr, String appId,
+      Set<String> unselectedFields) {
+    return getNextInterceptor().getApp(hsr, appId, unselectedFields);
+  }
+
+  @Override
+  public AppState getAppState(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return getNextInterceptor().getAppState(hsr, appId);
+  }
+
+  @Override
+  public Response updateAppState(AppState targetState, HttpServletRequest hsr,
+      String appId) throws AuthorizationException, YarnException,
+      InterruptedException, IOException {
+    return getNextInterceptor().updateAppState(targetState, hsr, appId);
+  }
+
+  @Override
+  public NodeToLabelsInfo getNodeToLabels(HttpServletRequest hsr)
+      throws IOException {
+    return getNextInterceptor().getNodeToLabels(hsr);
+  }
+
+  @Override
+  public LabelsToNodesInfo getLabelsToNodes(Set<String> labels)
+      throws IOException {
+    return getNextInterceptor().getLabelsToNodes(labels);
+  }
+
+  @Override
+  public Response replaceLabelsOnNodes(NodeToLabelsEntryList newNodeToLabels,
+      HttpServletRequest hsr) throws Exception {
+    return getNextInterceptor().replaceLabelsOnNodes(newNodeToLabels, hsr);
+  }
+
+  @Override
+  public Response replaceLabelsOnNode(Set<String> newNodeLabelsName,
+      HttpServletRequest hsr, String nodeId) throws Exception {
+    return getNextInterceptor().replaceLabelsOnNode(newNodeLabelsName, hsr,
+        nodeId);
+  }
+
+  @Override
+  public NodeLabelsInfo getClusterNodeLabels(HttpServletRequest hsr)
+      throws IOException {
+    return getNextInterceptor().getClusterNodeLabels(hsr);
+  }
+
+  @Override
+  public Response addToClusterNodeLabels(NodeLabelsInfo newNodeLabels,
+      HttpServletRequest hsr) throws Exception {
+    return getNextInterceptor().addToClusterNodeLabels(newNodeLabels, hsr);
+  }
+
+  @Override
+  public Response removeFromCluserNodeLabels(Set<String> oldNodeLabels,
+      HttpServletRequest hsr) throws Exception {
+    return getNextInterceptor().removeFromCluserNodeLabels(oldNodeLabels, hsr);
+  }
+
+  @Override
+  public NodeLabelsInfo getLabelsOnNode(HttpServletRequest hsr, String nodeId)
+      throws IOException {
+    return getNextInterceptor().getLabelsOnNode(hsr, nodeId);
+  }
+
+  @Override
+  public AppPriority getAppPriority(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return getNextInterceptor().getAppPriority(hsr, appId);
+  }
+
+  @Override
+  public Response updateApplicationPriority(AppPriority targetPriority,
+      HttpServletRequest hsr, String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return getNextInterceptor().updateApplicationPriority(targetPriority, hsr,
+        appId);
+  }
+
+  @Override
+  public AppQueue getAppQueue(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return getNextInterceptor().getAppQueue(hsr, appId);
+  }
+
+  @Override
+  public Response updateAppQueue(AppQueue targetQueue, HttpServletRequest hsr,
+      String appId) throws AuthorizationException, YarnException,
+      InterruptedException, IOException {
+    return getNextInterceptor().updateAppQueue(targetQueue, hsr, appId);
+  }
+
+  @Override
+  public Response createNewApplication(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return getNextInterceptor().createNewApplication(hsr);
+  }
+
+  @Override
+  public Response submitApplication(ApplicationSubmissionContextInfo newApp,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return getNextInterceptor().submitApplication(newApp, hsr);
+  }
+
+  @Override
+  public Response postDelegationToken(DelegationToken tokenData,
+      HttpServletRequest hsr) throws AuthorizationException, IOException,
+      InterruptedException, Exception {
+    return getNextInterceptor().postDelegationToken(tokenData, hsr);
+  }
+
+  @Override
+  public Response postDelegationTokenExpiration(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, Exception {
+    return getNextInterceptor().postDelegationTokenExpiration(hsr);
+  }
+
+  @Override
+  public Response cancelDelegationToken(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    return getNextInterceptor().cancelDelegationToken(hsr);
+  }
+
+  @Override
+  public Response createNewReservation(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return getNextInterceptor().createNewReservation(hsr);
+  }
+
+  @Override
+  public Response submitReservation(ReservationSubmissionRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return getNextInterceptor().submitReservation(resContext, hsr);
+  }
+
+  @Override
+  public Response updateReservation(ReservationUpdateRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return getNextInterceptor().updateReservation(resContext, hsr);
+  }
+
+  @Override
+  public Response deleteReservation(ReservationDeleteRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return getNextInterceptor().deleteReservation(resContext, hsr);
+  }
+
+  @Override
+  public Response listReservation(String queue, String reservationId,
+      long startTime, long endTime, boolean includeResourceAllocations,
+      HttpServletRequest hsr) throws Exception {
+    return getNextInterceptor().listReservation(queue, reservationId, startTime,
+        endTime, includeResourceAllocations, hsr);
+  }
+
+  @Override
+  public AppTimeoutInfo getAppTimeout(HttpServletRequest hsr, String appId,
+      String type) throws AuthorizationException {
+    return getNextInterceptor().getAppTimeout(hsr, appId, type);
+  }
+
+  @Override
+  public AppTimeoutsInfo getAppTimeouts(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return getNextInterceptor().getAppTimeouts(hsr, appId);
+  }
+
+  @Override
+  public Response updateApplicationTimeout(AppTimeoutInfo appTimeout,
+      HttpServletRequest hsr, String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return getNextInterceptor().updateApplicationTimeout(appTimeout, hsr,
+        appId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServices.java
new file mode 100644
index 0000000..c96575c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServices.java
@@ -0,0 +1,269 @@
+/**
+ * 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.router.webapp;
+
+import java.io.IOException;
+import java.util.Map;
+
+import javax.ws.rs.core.Response;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
+import org.apache.hadoop.yarn.server.router.webapp.RouterWebServices.RequestInterceptorChainWrapper;
+import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class to validate the WebService interceptor model inside the Router.
+ */
+public class TestRouterWebServices extends BaseRouterWebServicesTest {
+
+  private String user = "test1";
+
+  /**
+   * Test that all requests get forwarded to the last interceptor in the chain
+   * get back the responses.
+   */
+  @Test
+  public void testRouterWebServicesE2E() throws Exception {
+
+    ClusterInfo clusterInfo = get(user);
+    Assert.assertNotNull(clusterInfo);
+
+    ClusterInfo clusterInfo2 = getClusterInfo(user);
+    Assert.assertNotNull(clusterInfo2);
+
+    ClusterMetricsInfo clusterMetricsInfo = getClusterMetricsInfo(user);
+    Assert.assertNotNull(clusterMetricsInfo);
+
+    SchedulerTypeInfo schedulerTypeInfo = getSchedulerInfo(user);
+    Assert.assertNotNull(schedulerTypeInfo);
+
+    String dumpResult = dumpSchedulerLogs(user);
+    Assert.assertNotNull(dumpResult);
+
+    NodesInfo nodesInfo = getNodes(user);
+    Assert.assertNotNull(nodesInfo);
+
+    NodeInfo nodeInfo = getNode(user);
+    Assert.assertNotNull(nodeInfo);
+
+    AppsInfo appsInfo = getApps(user);
+    Assert.assertNotNull(appsInfo);
+
+    ActivitiesInfo activitiesInfo = getActivities(user);
+    Assert.assertNotNull(activitiesInfo);
+
+    AppActivitiesInfo appActiviesInfo = getAppActivities(user);
+    Assert.assertNotNull(appActiviesInfo);
+
+    ApplicationStatisticsInfo applicationStatisticsInfo =
+        getAppStatistics(user);
+    Assert.assertNotNull(applicationStatisticsInfo);
+
+    AppInfo appInfo = getApp(user);
+    Assert.assertNotNull(appInfo);
+
+    AppState appState = getAppState(user);
+    Assert.assertNotNull(appState);
+
+    Response response = updateAppState(user);
+    Assert.assertNotNull(response);
+
+    NodeToLabelsInfo nodeToLabelsInfo = getNodeToLabels(user);
+    Assert.assertNotNull(nodeToLabelsInfo);
+
+    LabelsToNodesInfo labelsToNodesInfo = getLabelsToNodes(user);
+    Assert.assertNotNull(labelsToNodesInfo);
+
+    Response response2 = replaceLabelsOnNodes(user);
+    Assert.assertNotNull(response2);
+
+    Response response3 = replaceLabelsOnNode(user);
+    Assert.assertNotNull(response3);
+
+    NodeLabelsInfo nodeLabelsInfo = getClusterNodeLabels(user);
+    Assert.assertNotNull(nodeLabelsInfo);
+
+    Response response4 = addToClusterNodeLabels(user);
+    Assert.assertNotNull(response4);
+
+    Response response5 = removeFromCluserNodeLabels(user);
+    Assert.assertNotNull(response5);
+
+    NodeLabelsInfo nodeLabelsInfo2 = getLabelsOnNode(user);
+    Assert.assertNotNull(nodeLabelsInfo2);
+
+    AppPriority appPriority = getAppPriority(user);
+    Assert.assertNotNull(appPriority);
+
+    Response response6 = updateApplicationPriority(user);
+    Assert.assertNotNull(response6);
+
+    AppQueue appQueue = getAppQueue(user);
+    Assert.assertNotNull(appQueue);
+
+    Response response7 = updateAppQueue(user);
+    Assert.assertNotNull(response7);
+
+    Response response8 = createNewApplication(user);
+    Assert.assertNotNull(response8);
+
+    Response response9 = submitApplication(user);
+    Assert.assertNotNull(response9);
+
+    Response response10 = postDelegationToken(user);
+    Assert.assertNotNull(response10);
+
+    Response response11 = postDelegationTokenExpiration(user);
+    Assert.assertNotNull(response11);
+
+    Response response12 = cancelDelegationToken(user);
+    Assert.assertNotNull(response12);
+
+    Response response13 = createNewReservation(user);
+    Assert.assertNotNull(response13);
+
+    Response response14 = submitReservation(user);
+    Assert.assertNotNull(response14);
+
+    Response response15 = updateReservation(user);
+    Assert.assertNotNull(response15);
+
+    Response response16 = deleteReservation(user);
+    Assert.assertNotNull(response16);
+
+    Response response17 = listReservation(user);
+    Assert.assertNotNull(response17);
+
+    AppTimeoutInfo appTimeoutInfo = getAppTimeout(user);
+    Assert.assertNotNull(appTimeoutInfo);
+
+    AppTimeoutsInfo appTimeoutsInfo = getAppTimeouts(user);
+    Assert.assertNotNull(appTimeoutsInfo);
+
+    Response response18 = updateApplicationTimeout(user);
+    Assert.assertNotNull(response18);
+
+    AppAttemptsInfo appAttemptsInfo = getAppAttempts(user);
+    Assert.assertNotNull(appAttemptsInfo);
+
+    AppAttemptInfo appAttemptInfo = getAppAttempt(user);
+    Assert.assertNotNull(appAttemptInfo);
+
+    ContainersInfo containersInfo = getContainers(user);
+    Assert.assertNotNull(containersInfo);
+
+    ContainerInfo containerInfo = getContainer(user);
+    Assert.assertNotNull(containerInfo);
+  }
+
+  /**
+   * Tests if the pipeline is created properly.
+   */
+  @Test
+  public void testRequestInterceptorChainCreation() throws Exception {
+    RESTRequestInterceptor root =
+        super.getRouterWebServices().createRequestInterceptorChain();
+    int index = 0;
+    while (root != null) {
+      // The current pipeline is:
+      // PassThroughRESTRequestInterceptor - index = 0
+      // PassThroughRESTRequestInterceptor - index = 1
+      // PassThroughRESTRequestInterceptor - index = 2
+      // MockRESTRequestInterceptor - index = 3
+      switch (index) {
+      case 0: // Fall to the next case
+      case 1: // Fall to the next case
+      case 2:
+        // If index is equal to 0,1 or 2 we fall in this check
+        Assert.assertEquals(PassThroughRESTRequestInterceptor.class.getName(),
+            root.getClass().getName());
+        break;
+      case 3:
+        Assert.assertEquals(MockRESTRequestInterceptor.class.getName(),
+            root.getClass().getName());
+        break;
+      default:
+        Assert.fail();
+      }
+      root = root.getNextInterceptor();
+      index++;
+    }
+    Assert.assertEquals("The number of interceptors in chain does not match", 4,
+        index);
+  }
+
+  /**
+   * Test if the different chains for users are generated, and LRU cache is
+   * working as expected.
+   */
+  @Test
+  public void testUsersChainMapWithLRUCache()
+      throws YarnException, IOException, InterruptedException {
+    getInterceptorChain("test1");
+    getInterceptorChain("test2");
+    getInterceptorChain("test3");
+    getInterceptorChain("test4");
+    getInterceptorChain("test5");
+    getInterceptorChain("test6");
+    getInterceptorChain("test7");
+    getInterceptorChain("test8");
+
+    Map<String, RequestInterceptorChainWrapper> pipelines =
+        getRouterWebServices().getPipelines();
+    Assert.assertEquals(8, pipelines.size());
+
+    getInterceptorChain("test9");
+    getInterceptorChain("test10");
+    getInterceptorChain("test1");
+    getInterceptorChain("test11");
+
+    // The cache max size is defined in TEST_MAX_CACHE_SIZE
+    Assert.assertEquals(10, pipelines.size());
+
+    RequestInterceptorChainWrapper chain = pipelines.get("test1");
+    Assert.assertNotNull("test1 should not be evicted", chain);
+
+    chain = pipelines.get("test2");
+    Assert.assertNull("test2 should have been evicted", chain);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java
new file mode 100644
index 0000000..d7b1a0f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java
@@ -0,0 +1,1298 @@
+/**
+ * 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.router.webapp;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServiceProtocol;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewApplication;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewReservation;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
+import org.apache.hadoop.yarn.server.router.Router;
+import org.apache.hadoop.yarn.server.webapp.WebServices;
+import org.apache.hadoop.yarn.server.webapp.dao.AppsInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+import org.codehaus.jettison.json.JSONException;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.WebResource.Builder;
+
+import net.jcip.annotations.NotThreadSafe;
+
+/**
+ * This test validate E2E the correctness of the RouterWebServices. It starts
+ * Router, RM and NM in 3 different processes to avoid servlet conflicts. Each
+ * test creates a REST call to Router and validate that the operation complete
+ * successfully.
+ */
+@NotThreadSafe
+public class TestRouterWebServicesREST {
+
+  private static String userName = "test";
+
+  private static JavaProcess rm;
+  private static JavaProcess nm;
+  private static JavaProcess router;
+
+  private static Configuration conf;
+
+  private static final int STATUS_OK = 200;
+  private static final int STATUS_ACCEPTED = 202;
+  private static final int STATUS_BADREQUEST = 400;
+  private static final int STATUS_ERROR = 500;
+
+  /**
+   * Wait until the webservice is up and running.
+   */
+  private static void waitWebAppRunning(String address, String path) {
+    while (true) {
+      Client clientToRouter = Client.create();
+      WebResource toRouter = clientToRouter.resource(address).path(path);
+      try {
+        ClientResponse response = toRouter.accept(MediaType.APPLICATION_JSON)
+            .get(ClientResponse.class);
+        if (response.getStatus() == STATUS_OK) {
+          // process is up and running
+          return;
+        }
+      } catch (ClientHandlerException e) {
+        // process is not up and running
+        continue;
+      }
+    }
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    conf = new YarnConfiguration();
+    rm = new JavaProcess(ResourceManager.class);
+    router = new JavaProcess(Router.class);
+    nm = new JavaProcess(NodeManager.class);
+
+    // The tests cannot start if all the service are not up and running.
+    waitWebAppRunning(WebAppUtils.getRMWebAppURLWithScheme(conf),
+        RMWSConsts.RM_WEB_SERVICE_PATH);
+
+    waitWebAppRunning(WebAppUtils.getRouterWebAppURLWithScheme(conf),
+        RMWSConsts.RM_WEB_SERVICE_PATH);
+
+    waitWebAppRunning("http://" + WebAppUtils.getNMWebAppURLWithoutScheme(conf),
+        "/ws/v1/node");
+  }
+
+  @AfterClass
+  public static void stop() throws Exception {
+    nm.stop();
+    router.stop();
+    rm.stop();
+  }
+
+  /**
+   * Performs 2 GET calls one to RM and the one to Router. In positive case, it
+   * returns the 2 answers in a list.
+   */
+  private static <T> List<T> performGetCalls(String path, Class<T> returnType,
+      String queryName, String queryValue)
+      throws IOException, InterruptedException {
+    Client clientToRouter = Client.create();
+    WebResource toRouter = clientToRouter
+        .resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)).path(path);
+
+    Client clientToRM = Client.create();
+    WebResource toRM = clientToRM
+        .resource(WebAppUtils.getRMWebAppURLWithScheme(conf)).path(path);
+
+    Builder toRouterBuilder;
+    Builder toRMBuilder;
+
+    if (queryValue != null && queryName != null) {
+      toRouterBuilder = toRouter.queryParam(queryName, queryValue)
+          .accept(MediaType.APPLICATION_XML);
+      toRMBuilder = toRM.queryParam(queryName, queryValue)
+          .accept(MediaType.APPLICATION_XML);
+    } else {
+      toRouterBuilder = toRouter.accept(MediaType.APPLICATION_XML);
+      toRMBuilder = toRM.accept(MediaType.APPLICATION_XML);
+    }
+
+    return UserGroupInformation.createRemoteUser(userName)
+        .doAs(new PrivilegedExceptionAction<List<T>>() {
+          @Override
+          public List<T> run() throws Exception {
+            ClientResponse response = toRouterBuilder.get(ClientResponse.class);
+            ClientResponse response2 = toRMBuilder.get(ClientResponse.class);
+            if (response.getStatus() == STATUS_OK
+                && response2.getStatus() == STATUS_OK) {
+              List<T> responses = new ArrayList<T>();
+              responses.add(response.getEntity(returnType));
+              responses.add(response2.getEntity(returnType));
+              return responses;
+            } else {
+              Assert.fail();
+            }
+            return null;
+          }
+        });
+  }
+
+  /**
+   * Performs a POST/PUT/DELETE call to Router and returns the ClientResponse.
+   */
+  private static ClientResponse performCall(String webAddress, String queryKey,
+      String queryValue, Object context, HTTPMethods method)
+      throws IOException, InterruptedException {
+
+    return UserGroupInformation.createRemoteUser(userName)
+        .doAs(new PrivilegedExceptionAction<ClientResponse>() {
+          @Override
+          public ClientResponse run() throws Exception {
+            Client clientToRouter = Client.create();
+            WebResource toRouter = clientToRouter
+                .resource(WebAppUtils.getRouterWebAppURLWithScheme(conf))
+                .path(webAddress);
+
+            WebResource toRouterWR;
+            if (queryKey != null && queryValue != null) {
+              toRouterWR = toRouter.queryParam(queryKey, queryValue);
+            } else {
+              toRouterWR = toRouter;
+            }
+
+            Builder builder = null;
+            if (context != null) {
+              builder = toRouterWR.entity(context, MediaType.APPLICATION_JSON);
+              builder = builder.accept(MediaType.APPLICATION_JSON);
+            } else {
+              builder = toRouter.accept(MediaType.APPLICATION_JSON);
+            }
+
+            ClientResponse response = null;
+
+            switch (method) {
+            case DELETE:
+              response = builder.delete(ClientResponse.class);
+              break;
+            case POST:
+              response = builder.post(ClientResponse.class);
+              break;
+            case PUT:
+              response = builder.put(ClientResponse.class);
+              break;
+            default:
+              break;
+            }
+
+            return response;
+          }
+        });
+  }
+
+  /**
+   * This test validates the correctness of {@link RMWebServiceProtocol#get()}
+   * inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testInfoXML() throws JSONException, Exception {
+
+    List<ClusterInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH, ClusterInfo.class, null, null);
+
+    ClusterInfo routerResponse = responses.get(0);
+    ClusterInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getRMVersion(),
+        routerResponse.getRMVersion());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getClusterInfo()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testClusterInfoXML() throws JSONException, Exception {
+
+    List<ClusterInfo> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.INFO,
+            ClusterInfo.class, null, null);
+
+    ClusterInfo routerResponse = responses.get(0);
+    ClusterInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getRMVersion(),
+        routerResponse.getRMVersion());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getClusterMetricsInfo()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testMetricsInfoXML() throws JSONException, Exception {
+
+    List<ClusterMetricsInfo> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.METRICS,
+            ClusterMetricsInfo.class, null, null);
+
+    ClusterMetricsInfo routerResponse = responses.get(0);
+    ClusterMetricsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getActiveNodes(),
+        routerResponse.getActiveNodes());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getSchedulerInfo()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testSchedulerInfoXML() throws JSONException, Exception {
+
+    List<SchedulerTypeInfo> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER,
+            SchedulerTypeInfo.class, null, null);
+
+    SchedulerTypeInfo routerResponse = responses.get(0);
+    SchedulerTypeInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getSchedulerInfo().getSchedulerType(),
+        routerResponse.getSchedulerInfo().getSchedulerType());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getNodes()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testNodesXML() throws JSONException, Exception {
+
+    List<NodesInfo> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES,
+            NodesInfo.class, RMWSConsts.STATES, "LOST");
+
+    NodesInfo routerResponse = responses.get(0);
+    NodesInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getNodes().size(),
+        routerResponse.getNodes().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getNode()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testNodeXML() throws JSONException, Exception {
+
+    List<NodeInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES + "/" + getNodeId(),
+        NodeInfo.class, null, null);
+
+    NodeInfo routerResponse = responses.get(0);
+    NodeInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getVersion(), routerResponse.getVersion());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getActivities()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testActiviesXML() throws JSONException, Exception {
+
+    List<ActivitiesInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_ACTIVITIES,
+        ActivitiesInfo.class, null, null);
+
+    ActivitiesInfo routerResponse = responses.get(0);
+    ActivitiesInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppActivities()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppActivitiesXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppActivitiesInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_APP_ACTIVITIES,
+        AppActivitiesInfo.class, RMWSConsts.APP_ID, appId);
+
+    AppActivitiesInfo routerResponse = responses.get(0);
+    AppActivitiesInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppStatistics()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppStatisticsXML() throws JSONException, Exception {
+
+    submitApplication();
+
+    List<ApplicationStatisticsInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APP_STATISTICS,
+        ApplicationStatisticsInfo.class, RMWSConsts.STATES, "RUNNING");
+
+    ApplicationStatisticsInfo routerResponse = responses.get(0);
+    ApplicationStatisticsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getStatItems().size(),
+        routerResponse.getStatItems().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#dumpSchedulerLogs()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testDumpSchedulerLogsXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_LOGS,
+            null, null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    ClientResponse response =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_LOGS,
+            RMWSConsts.TIME, "1", null, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_BADREQUEST) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#createNewApplication()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testNewApplicationXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS_NEW_APPLICATION, null,
+        null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS_NEW_APPLICATION, null,
+        null, null, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_OK) {
+      NewApplication ci = response.getEntity(NewApplication.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#submitApplication()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testSubmitApplicationXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, null,
+            null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    context.setApplicationId(getNewApplicationId().getApplicationId());
+
+    ClientResponse response =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, null,
+            null, context, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_ACCEPTED) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getApps()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppsXML() throws JSONException, Exception {
+
+    submitApplication();
+
+    List<AppsInfo> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS,
+            AppsInfo.class, null, null);
+
+    AppsInfo routerResponse = responses.get(0);
+    AppsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getApps().size(),
+        rmResponse.getApps().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getApp()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId,
+        AppInfo.class, null, null);
+
+    AppInfo routerResponse = responses.get(0);
+    AppInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getAMHostHttpAddress(),
+        rmResponse.getAMHostHttpAddress());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppAttempts()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppAttemptXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppAttemptsInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+            + RMWSConsts.ATTEMPTS,
+        AppAttemptsInfo.class, null, null);
+
+    AppAttemptsInfo routerResponse = responses.get(0);
+    AppAttemptsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getAttempts().size(),
+        rmResponse.getAttempts().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppState()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppStateXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppState> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/"
+            + appId + "/" + RMWSConsts.STATE, AppState.class, null, null);
+
+    AppState routerResponse = responses.get(0);
+    AppState rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getState(), rmResponse.getState());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#updateAppState()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testUpdateAppStateXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(RMWSConsts.RM_WEB_SERVICE_PATH
+        + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.STATE, null, null,
+        null, HTTPMethods.POST);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    AppState appState = new AppState("KILLED");
+
+    ClientResponse response = performCall(RMWSConsts.RM_WEB_SERVICE_PATH
+        + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.STATE, null, null,
+        appState, HTTPMethods.PUT);
+
+    if (response.getStatus() == STATUS_ACCEPTED) {
+      AppState ci = response.getEntity(AppState.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppPriority()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppPriorityXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppPriority> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/"
+            + appId + "/" + RMWSConsts.PRIORITY, AppPriority.class, null, null);
+
+    AppPriority routerResponse = responses.get(0);
+    AppPriority rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getPriority(), rmResponse.getPriority());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#updateApplicationPriority()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testUpdateAppPriorityXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(RMWSConsts.RM_WEB_SERVICE_PATH
+        + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.PRIORITY, null, null,
+        null, HTTPMethods.POST);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    AppPriority appPriority = new AppPriority(1);
+
+    ClientResponse response =
+        performCall(
+            RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+                + RMWSConsts.PRIORITY,
+            null, null, appPriority, HTTPMethods.PUT);
+
+    if (response.getStatus() == STATUS_OK) {
+      AppPriority ci = response.getEntity(AppPriority.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppQueue()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppQueueXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppQueue> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/"
+            + appId + "/" + RMWSConsts.QUEUE, AppQueue.class, null, null);
+
+    AppQueue routerResponse = responses.get(0);
+    AppQueue rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getQueue(), rmResponse.getQueue());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#updateAppQueue()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testUpdateAppQueueXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(RMWSConsts.RM_WEB_SERVICE_PATH
+        + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.QUEUE, null, null,
+        null, HTTPMethods.POST);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    AppQueue appQueue = new AppQueue("default");
+
+    ClientResponse response = performCall(RMWSConsts.RM_WEB_SERVICE_PATH
+        + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.QUEUE, null, null,
+        appQueue, HTTPMethods.PUT);
+
+    if (response.getStatus() == STATUS_OK) {
+      AppQueue ci = response.getEntity(AppQueue.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppTimeouts()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppTimeoutsXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppTimeoutsInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+            + RMWSConsts.TIMEOUTS,
+        AppTimeoutsInfo.class, null, null);
+
+    AppTimeoutsInfo routerResponse = responses.get(0);
+    AppTimeoutsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getAppTimeouts().size(),
+        rmResponse.getAppTimeouts().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppTimeout()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppTimeoutXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppTimeoutInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+            + RMWSConsts.TIMEOUTS + "/" + "LIFETIME",
+        AppTimeoutInfo.class, null, null);
+
+    AppTimeoutInfo routerResponse = responses.get(0);
+    AppTimeoutInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getExpireTime(), rmResponse.getExpireTime());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#updateApplicationTimeout()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testUpdateAppTimeoutsXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(RMWSConsts.RM_WEB_SERVICE_PATH
+        + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.TIMEOUT, null, null,
+        null, HTTPMethods.POST);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    // Create a bad request
+    AppTimeoutInfo appTimeoutInfo = new AppTimeoutInfo();
+
+    ClientResponse response =
+        performCall(
+            RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+                + RMWSConsts.TIMEOUT,
+            null, null, appTimeoutInfo, HTTPMethods.PUT);
+
+    if (response.getStatus() == STATUS_BADREQUEST) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#createNewReservation()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testNewReservationXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_NEW,
+            null, null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    ClientResponse response =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_NEW,
+            null, null, null, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_OK) {
+      NewReservation ci = response.getEntity(NewReservation.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#submitReservation()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testSubmitReservationXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_SUBMIT, null,
+        null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    ReservationSubmissionRequestInfo context =
+        new ReservationSubmissionRequestInfo();
+    context.setReservationId(getNewReservationId().getReservationId());
+    // ReservationDefinition is null
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_SUBMIT, null,
+        null, context, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_BADREQUEST) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#updateReservation()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testUpdateReservationXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_UPDATE, null,
+        null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    String reservationId = getNewReservationId().getReservationId();
+    ReservationUpdateRequestInfo context = new ReservationUpdateRequestInfo();
+    context.setReservationId(reservationId);
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_UPDATE, null,
+        null, context, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_BADREQUEST) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#deleteReservation()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testDeleteReservationXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_DELETE, null,
+        null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    String reservationId = getNewReservationId().getReservationId();
+    ReservationDeleteRequestInfo context = new ReservationDeleteRequestInfo();
+    context.setReservationId(reservationId);
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_DELETE, null,
+        null, context, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_BADREQUEST) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getNodeToLabels()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testGetNodeToLabelsXML() throws JSONException, Exception {
+
+    List<NodeToLabelsInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.GET_NODE_TO_LABELS,
+        NodeToLabelsInfo.class, null, null);
+
+    NodeToLabelsInfo routerResponse = responses.get(0);
+    NodeToLabelsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getNodeToLabels().size(),
+        rmResponse.getNodeToLabels().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getClusterNodeLabels()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testGetClusterNodeLabelsXML() throws JSONException, Exception {
+
+    List<NodeLabelsInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.GET_NODE_LABELS,
+        NodeLabelsInfo.class, null, null);
+
+    NodeLabelsInfo routerResponse = responses.get(0);
+    NodeLabelsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getNodeLabels().size(),
+        rmResponse.getNodeLabels().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getLabelsOnNode()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testGetLabelsOnNodeXML() throws JSONException, Exception {
+
+    List<NodeLabelsInfo> responses =
+        performGetCalls(
+            RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES + "/"
+                + getNodeId() + "/" + RMWSConsts.GET_LABELS,
+            NodeLabelsInfo.class, null, null);
+
+    NodeLabelsInfo routerResponse = responses.get(0);
+    NodeLabelsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getNodeLabels().size(),
+        rmResponse.getNodeLabels().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getLabelsToNodes()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testGetLabelsMappingXML() throws JSONException, Exception {
+
+    List<LabelsToNodesInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.LABEL_MAPPINGS,
+        LabelsToNodesInfo.class, null, null);
+
+    LabelsToNodesInfo routerResponse = responses.get(0);
+    LabelsToNodesInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getLabelsToNodes().size(),
+        rmResponse.getLabelsToNodes().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#addToClusterNodeLabels()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAddToClusterNodeLabelsXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.ADD_NODE_LABELS,
+            null, null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    List<NodeLabel> nodeLabels = new ArrayList<NodeLabel>();
+    nodeLabels.add(NodeLabel.newInstance("default"));
+    NodeLabelsInfo context = new NodeLabelsInfo(nodeLabels);
+
+    ClientResponse response =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.ADD_NODE_LABELS,
+            null, null, context, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_OK) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#removeFromCluserNodeLabels()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testRemoveFromCluserNodeLabelsXML()
+      throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.REMOVE_NODE_LABELS, null,
+        null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    addNodeLabel();
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.REMOVE_NODE_LABELS,
+        RMWSConsts.LABELS, "default", null, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_OK) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#replaceLabelsOnNodes()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testReplaceLabelsOnNodesXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.REPLACE_NODE_TO_LABELS,
+        null, null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    addNodeLabel();
+
+    NodeToLabelsEntryList context = new NodeToLabelsEntryList();
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.REPLACE_NODE_TO_LABELS,
+        null, null, context, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_OK) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#replaceLabelsOnNode()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testReplaceLabelsOnNodeXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse =
+        performCall(
+            RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES + "/"
+                + getNodeId() + "/replace-labels",
+            null, null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    addNodeLabel();
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES + "/" + getNodeId()
+            + "/replace-labels",
+        RMWSConsts.LABELS, "default", null, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_OK) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of {@link WebServices#getAppAttempt}
+   * inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testGetAppAttemptXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppAttemptInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+            + RMWSConsts.APPATTEMPTS + "/" + getAppAttempt(appId),
+        AppAttemptInfo.class, null, null);
+
+    AppAttemptInfo routerResponse = responses.get(0);
+    AppAttemptInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getAppAttemptId(),
+        rmResponse.getAppAttemptId());
+  }
+
+  /**
+   * This test validates the correctness of {@link WebServices#getContainers}
+   * inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testGetContainersXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<ContainersInfo> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/"
+            + appId + "/" + RMWSConsts.APPATTEMPTS + "/" + getAppAttempt(appId)
+            + "/" + RMWSConsts.CONTAINERS, ContainersInfo.class, null, null);
+
+    ContainersInfo routerResponse = responses.get(0);
+    ContainersInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getContainers().size(),
+        rmResponse.getContainers().size());
+  }
+
+  private String getNodeId() {
+    Client clientToRM = Client.create();
+    WebResource toRM =
+        clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf))
+            .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES);
+    ClientResponse response =
+        toRM.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
+    NodesInfo ci = response.getEntity(NodesInfo.class);
+    return ci.getNodes().get(0).getNodeId();
+  }
+
+  private NewApplication getNewApplicationId() {
+    Client clientToRM = Client.create();
+    WebResource toRM =
+        clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)).path(
+            RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS_NEW_APPLICATION);
+    ClientResponse response =
+        toRM.accept(MediaType.APPLICATION_XML).post(ClientResponse.class);
+    return response.getEntity(NewApplication.class);
+  }
+
+  private String submitApplication() {
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    String appId = getNewApplicationId().getApplicationId();
+    context.setApplicationId(appId);
+
+    Client clientToRouter = Client.create();
+    WebResource toRM =
+        clientToRouter.resource(WebAppUtils.getRMWebAppURLWithScheme(conf))
+            .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS);
+    toRM.entity(context, MediaType.APPLICATION_XML)
+        .accept(MediaType.APPLICATION_XML).post(ClientResponse.class);
+    return appId;
+  }
+
+  private NewReservation getNewReservationId() {
+    Client clientToRM = Client.create();
+    WebResource toRM =
+        clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf))
+            .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_NEW);
+    ClientResponse response =
+        toRM.accept(MediaType.APPLICATION_XML).post(ClientResponse.class);
+    return response.getEntity(NewReservation.class);
+  }
+
+  private String addNodeLabel() {
+    Client clientToRM = Client.create();
+    WebResource toRM =
+        clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf))
+            .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.ADD_NODE_LABELS);
+    List<NodeLabel> nodeLabels = new ArrayList<NodeLabel>();
+    nodeLabels.add(NodeLabel.newInstance("default"));
+    NodeLabelsInfo context = new NodeLabelsInfo(nodeLabels);
+    ClientResponse response = toRM.entity(context, MediaType.APPLICATION_XML)
+        .accept(MediaType.APPLICATION_XML).post(ClientResponse.class);
+    return response.getEntity(String.class);
+  }
+
+  private String getAppAttempt(String appId) {
+    Client clientToRM = Client.create();
+    WebResource toRM =
+        clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf))
+            .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId
+                + "/" + RMWSConsts.ATTEMPTS);
+    ClientResponse response =
+        toRM.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
+    AppAttemptsInfo ci = response.getEntity(AppAttemptsInfo.class);
+    return ci.getAttempts().get(0).getAppAttemptId();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/capacity-scheduler.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/capacity-scheduler.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/capacity-scheduler.xml
new file mode 100644
index 0000000..90c5eeb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/capacity-scheduler.xml
@@ -0,0 +1,111 @@
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<configuration>
+
+  <property>
+    <name>yarn.scheduler.capacity.maximum-applications</name>
+    <value>10000</value>
+    <description>
+      Maximum number of applications that can be pending and running.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.maximum-am-resource-percent</name>
+    <value>0.1</value>
+    <description>
+      Maximum percent of resources in the cluster which can be used to run
+      application masters i.e. controls number of concurrent running
+      applications.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.resource-calculator</name>
+    <value>org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator</value>
+    <description>
+      The ResourceCalculator implementation to be used to compare
+      Resources in the scheduler.
+      The default i.e. DefaultResourceCalculator only uses Memory while
+      DominantResourceCalculator uses dominant-resource to compare
+      multi-dimensional resources such as Memory, CPU etc.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.queues</name>
+    <value>default</value>
+    <description>
+      The queues at the this level (root is the root queue).
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.capacity</name>
+    <value>100</value>
+    <description>Default queue target capacity.</description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.user-limit-factor</name>
+    <value>1</value>
+    <description>
+      Default queue user limit a percentage from 0.0 to 1.0.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.maximum-capacity</name>
+    <value>100</value>
+    <description>
+      The maximum capacity of the default queue.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.state</name>
+    <value>RUNNING</value>
+    <description>
+      The state of the default queue. State can be one of RUNNING or STOPPED.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.acl_submit_applications</name>
+    <value>*</value>
+    <description>
+      The ACL of who can submit jobs to the default queue.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.acl_administer_queue</name>
+    <value>*</value>
+    <description>
+      The ACL of who can administer jobs on the default queue.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.node-locality-delay</name>
+    <value>-1</value>
+    <description>
+      Number of missed scheduling opportunities after which the CapacityScheduler
+      attempts to schedule rack-local containers.
+      Typically this should be set to number of racks in the cluster, this
+      feature is disabled by default, set to -1.
+    </description>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/log4j.properties
new file mode 100644
index 0000000..81a3f6a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/log4j.properties
@@ -0,0 +1,19 @@
+#   Licensed 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.
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml
new file mode 100644
index 0000000..f3e0de3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml
@@ -0,0 +1,30 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+<configuration>
+  <property>
+    <name>yarn.resourcemanager.reservation-system.enable</name>
+    <value>true</value>
+  </property>
+  <property>
+    <name>yarn.node-labels.enabled</name>
+    <value>true</value>
+  </property>
+  <property>
+    <name>yarn.resourcemanager.webapp.address</name>
+    <value>0.0.0.0:8080</value>
+  </property>
+</configuration>


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