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:06 UTC

[45/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/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java
new file mode 100644
index 0000000..bbb8326
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java
@@ -0,0 +1,876 @@
+/**
+ * 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.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.FormParam;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+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.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.router.Router;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
+import org.apache.hadoop.yarn.util.LRUCacheHashMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+
+/**
+ * RouterWebServices is a service that runs on each router that can be used to
+ * intercept and inspect {@link RMWebServiceProtocol} messages from client to
+ * the cluster resource manager. It listens {@link RMWebServiceProtocol} REST
+ * messages from the client and creates a request intercepting pipeline instance
+ * for each client. The pipeline is a chain of {@link RESTRequestInterceptor}
+ * instances that can inspect and modify the request/response as needed. The
+ * main difference with AMRMProxyService is the protocol they implement.
+ **/
+@Singleton
+@Path("/ws/v1/cluster")
+public class RouterWebServices implements RMWebServiceProtocol {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterWebServices.class);
+  private final Router router;
+  private final Configuration conf;
+  private @Context HttpServletResponse response;
+
+  private Map<String, RequestInterceptorChainWrapper> userPipelineMap;
+
+  // -------Default values of QueryParams for RMWebServiceProtocol--------
+
+  public static final String DEFAULT_QUEUE = "default";
+  public static final String DEFAULT_RESERVATION_ID = "";
+  public static final String DEFAULT_START_TIME = "0";
+  public static final String DEFAULT_END_TIME = "-1";
+  public static final String DEFAULT_INCLUDE_RESOURCE = "false";
+
+  @Inject
+  public RouterWebServices(final Router router, Configuration conf) {
+    this.router = router;
+    this.conf = conf;
+    int maxCacheSize =
+        conf.getInt(YarnConfiguration.ROUTER_PIPELINE_CACHE_MAX_SIZE,
+            YarnConfiguration.DEFAULT_ROUTER_PIPELINE_CACHE_MAX_SIZE);
+    this.userPipelineMap = Collections.synchronizedMap(
+        new LRUCacheHashMap<String, RequestInterceptorChainWrapper>(
+            maxCacheSize, true));
+  }
+
+  /**
+   * Returns the comma separated intercepter class names from the configuration.
+   *
+   * @param conf
+   * @return the intercepter class names as an instance of ArrayList
+   */
+  private List<String> getInterceptorClassNames(Configuration config) {
+    String configuredInterceptorClassNames =
+        config.get(YarnConfiguration.ROUTER_WEBAPP_INTERCEPTOR_CLASS_PIPELINE,
+            YarnConfiguration.DEFAULT_ROUTER_WEBAPP_INTERCEPTOR_CLASS);
+
+    List<String> interceptorClassNames = new ArrayList<String>();
+    Collection<String> tempList =
+        StringUtils.getStringCollection(configuredInterceptorClassNames);
+    for (String item : tempList) {
+      interceptorClassNames.add(item.trim());
+    }
+
+    return interceptorClassNames;
+  }
+
+  private void init() {
+    // clear content type
+    response.setContentType(null);
+  }
+
+  @VisibleForTesting
+  protected RequestInterceptorChainWrapper getInterceptorChain() {
+    String user = "";
+    try {
+      user = UserGroupInformation.getCurrentUser().getUserName();
+    } catch (IOException e) {
+      LOG.error("IOException " + e.getMessage());
+    }
+    if (!userPipelineMap.containsKey(user)) {
+      initializePipeline(user);
+    }
+    return userPipelineMap.get(user);
+  }
+
+  /**
+   * Gets the Request intercepter chains for all the users.
+   *
+   * @return the request intercepter chains.
+   */
+  @VisibleForTesting
+  protected Map<String, RequestInterceptorChainWrapper> getPipelines() {
+    return this.userPipelineMap;
+  }
+
+  /**
+   * This method creates and returns reference of the first intercepter in the
+   * chain of request intercepter instances.
+   *
+   * @return the reference of the first intercepter in the chain
+   */
+  @VisibleForTesting
+  protected RESTRequestInterceptor createRequestInterceptorChain() {
+
+    List<String> interceptorClassNames = getInterceptorClassNames(conf);
+
+    RESTRequestInterceptor pipeline = null;
+    RESTRequestInterceptor current = null;
+    for (String interceptorClassName : interceptorClassNames) {
+      try {
+        Class<?> interceptorClass = conf.getClassByName(interceptorClassName);
+        if (RESTRequestInterceptor.class.isAssignableFrom(interceptorClass)) {
+          RESTRequestInterceptor interceptorInstance =
+              (RESTRequestInterceptor) ReflectionUtils
+                  .newInstance(interceptorClass, conf);
+          if (pipeline == null) {
+            pipeline = interceptorInstance;
+            current = interceptorInstance;
+            continue;
+          } else {
+            current.setNextInterceptor(interceptorInstance);
+            current = interceptorInstance;
+          }
+        } else {
+          throw new YarnRuntimeException(
+              "Class: " + interceptorClassName + " not instance of "
+                  + RESTRequestInterceptor.class.getCanonicalName());
+        }
+      } catch (ClassNotFoundException e) {
+        throw new YarnRuntimeException(
+            "Could not instantiate RESTRequestInterceptor: "
+                + interceptorClassName,
+            e);
+      }
+    }
+
+    if (pipeline == null) {
+      throw new YarnRuntimeException(
+          "RequestInterceptor pipeline is not configured in the system");
+    }
+    return pipeline;
+  }
+
+  /**
+   * Initializes the request intercepter pipeline for the specified user.
+   *
+   * @param user
+   */
+  private void initializePipeline(String user) {
+    RequestInterceptorChainWrapper chainWrapper = null;
+    synchronized (this.userPipelineMap) {
+      if (this.userPipelineMap.containsKey(user)) {
+        LOG.info("Request to start an already existing user: {}"
+            + " was received, so ignoring.", user);
+        return;
+      }
+
+      chainWrapper = new RequestInterceptorChainWrapper();
+      this.userPipelineMap.put(user, chainWrapper);
+    }
+
+    // We register the pipeline instance in the map first and then initialize it
+    // later because chain initialization can be expensive and we would like to
+    // release the lock as soon as possible to prevent other applications from
+    // blocking when one application's chain is initializing
+    LOG.info("Initializing request processing pipeline for the user: {}", user);
+
+    try {
+      RESTRequestInterceptor interceptorChain =
+          this.createRequestInterceptorChain();
+      interceptorChain.init(user);
+      chainWrapper.init(interceptorChain);
+    } catch (Exception e) {
+      synchronized (this.userPipelineMap) {
+        this.userPipelineMap.remove(user);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Private structure for encapsulating RequestInterceptor and user instances.
+   *
+   */
+  @Private
+  public static class RequestInterceptorChainWrapper {
+    private RESTRequestInterceptor rootInterceptor;
+
+    /**
+     * Initializes the wrapper with the specified parameters.
+     *
+     * @param interceptor the first interceptor in the pipeline
+     */
+    public synchronized void init(RESTRequestInterceptor interceptor) {
+      this.rootInterceptor = interceptor;
+    }
+
+    /**
+     * Gets the root request intercepter.
+     *
+     * @return the root request intercepter
+     */
+    public synchronized RESTRequestInterceptor getRootInterceptor() {
+      return rootInterceptor;
+    }
+
+    /**
+     * Shutdown the chain of interceptors when the object is destroyed.
+     */
+    @Override
+    protected void finalize() {
+      rootInterceptor.shutdown();
+    }
+  }
+
+  @GET
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public ClusterInfo get() {
+    return getClusterInfo();
+  }
+
+  @GET
+  @Path(RMWSConsts.INFO)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public ClusterInfo getClusterInfo() {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getClusterInfo();
+  }
+
+  @GET
+  @Path(RMWSConsts.METRICS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public ClusterMetricsInfo getClusterMetricsInfo() {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getClusterMetricsInfo();
+  }
+
+  @GET
+  @Path(RMWSConsts.SCHEDULER)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public SchedulerTypeInfo getSchedulerInfo() {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getSchedulerInfo();
+  }
+
+  @POST
+  @Path(RMWSConsts.SCHEDULER_LOGS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public String dumpSchedulerLogs(@FormParam(RMWSConsts.TIME) String time,
+      @Context HttpServletRequest hsr) throws IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().dumpSchedulerLogs(time, hsr);
+  }
+
+  @GET
+  @Path(RMWSConsts.NODES)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public NodesInfo getNodes(@QueryParam(RMWSConsts.STATES) String states) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getNodes(states);
+  }
+
+  @GET
+  @Path(RMWSConsts.NODES_NODEID)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public NodeInfo getNode(@PathParam(RMWSConsts.NODEID) String nodeId) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getNode(nodeId);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppsInfo getApps(@Context HttpServletRequest hsr,
+      @QueryParam(RMWSConsts.STATE) String stateQuery,
+      @QueryParam(RMWSConsts.STATES) Set<String> statesQuery,
+      @QueryParam(RMWSConsts.FINAL_STATUS) String finalStatusQuery,
+      @QueryParam(RMWSConsts.USER) String userQuery,
+      @QueryParam(RMWSConsts.QUEUE) String queueQuery,
+      @QueryParam(RMWSConsts.LIMIT) String count,
+      @QueryParam(RMWSConsts.STARTED_TIME_BEGIN) String startedBegin,
+      @QueryParam(RMWSConsts.STARTED_TIME_END) String startedEnd,
+      @QueryParam(RMWSConsts.FINISHED_TIME_BEGIN) String finishBegin,
+      @QueryParam(RMWSConsts.FINISHED_TIME_END) String finishEnd,
+      @QueryParam(RMWSConsts.APPLICATION_TYPES) Set<String> applicationTypes,
+      @QueryParam(RMWSConsts.APPLICATION_TAGS) Set<String> applicationTags,
+      @QueryParam(RMWSConsts.DESELECTS) Set<String> unselectedFields) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getApps(hsr, stateQuery, statesQuery,
+        finalStatusQuery, userQuery, queueQuery, count, startedBegin,
+        startedEnd, finishBegin, finishEnd, applicationTypes, applicationTags,
+        unselectedFields);
+  }
+
+  @GET
+  @Path(RMWSConsts.SCHEDULER_ACTIVITIES)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public ActivitiesInfo getActivities(@Context HttpServletRequest hsr,
+      @QueryParam(RMWSConsts.NODEID) String nodeId) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getActivities(hsr, nodeId);
+  }
+
+  @GET
+  @Path(RMWSConsts.SCHEDULER_APP_ACTIVITIES)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppActivitiesInfo getAppActivities(@Context HttpServletRequest hsr,
+      @QueryParam(RMWSConsts.APP_ID) String appId,
+      @QueryParam(RMWSConsts.MAX_TIME) String time) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppActivities(hsr, appId, time);
+  }
+
+  @GET
+  @Path(RMWSConsts.APP_STATISTICS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public ApplicationStatisticsInfo getAppStatistics(
+      @Context HttpServletRequest hsr,
+      @QueryParam(RMWSConsts.STATES) Set<String> stateQueries,
+      @QueryParam(RMWSConsts.APPLICATION_TYPES) Set<String> typeQueries) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppStatistics(hsr, stateQueries,
+        typeQueries);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppInfo getApp(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId,
+      @QueryParam(RMWSConsts.DESELECTS) Set<String> unselectedFields) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getApp(hsr, appId, unselectedFields);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID_STATE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppState getAppState(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppState(hsr, appId);
+  }
+
+  @PUT
+  @Path(RMWSConsts.APPS_APPID_STATE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response updateAppState(AppState targetState,
+      @Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().updateAppState(targetState, hsr,
+        appId);
+  }
+
+  @GET
+  @Path(RMWSConsts.GET_NODE_TO_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public NodeToLabelsInfo getNodeToLabels(@Context HttpServletRequest hsr)
+      throws IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getNodeToLabels(hsr);
+  }
+
+  @GET
+  @Path(RMWSConsts.LABEL_MAPPINGS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public LabelsToNodesInfo getLabelsToNodes(
+      @QueryParam(RMWSConsts.LABELS) Set<String> labels) throws IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getLabelsToNodes(labels);
+  }
+
+  @POST
+  @Path(RMWSConsts.REPLACE_NODE_TO_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response replaceLabelsOnNodes(
+      final NodeToLabelsEntryList newNodeToLabels,
+      @Context HttpServletRequest hsr) throws Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().replaceLabelsOnNodes(newNodeToLabels,
+        hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.NODES_NODEID_REPLACE_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response replaceLabelsOnNode(
+      @QueryParam(RMWSConsts.LABELS) Set<String> newNodeLabelsName,
+      @Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.NODEID) String nodeId) throws Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().replaceLabelsOnNode(newNodeLabelsName,
+        hsr, nodeId);
+  }
+
+  @GET
+  @Path(RMWSConsts.GET_NODE_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public NodeLabelsInfo getClusterNodeLabels(@Context HttpServletRequest hsr)
+      throws IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getClusterNodeLabels(hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.ADD_NODE_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response addToClusterNodeLabels(NodeLabelsInfo newNodeLabels,
+      @Context HttpServletRequest hsr) throws Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().addToClusterNodeLabels(newNodeLabels,
+        hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.REMOVE_NODE_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response removeFromCluserNodeLabels(
+      @QueryParam(RMWSConsts.LABELS) Set<String> oldNodeLabels,
+      @Context HttpServletRequest hsr) throws Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor()
+        .removeFromCluserNodeLabels(oldNodeLabels, hsr);
+  }
+
+  @GET
+  @Path(RMWSConsts.NODES_NODEID_GETLABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public NodeLabelsInfo getLabelsOnNode(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.NODEID) String nodeId) throws IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getLabelsOnNode(hsr, nodeId);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID_PRIORITY)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppPriority getAppPriority(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppPriority(hsr, appId);
+  }
+
+  @PUT
+  @Path(RMWSConsts.APPS_APPID_PRIORITY)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response updateApplicationPriority(AppPriority targetPriority,
+      @Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor()
+        .updateApplicationPriority(targetPriority, hsr, appId);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID_QUEUE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppQueue getAppQueue(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppQueue(hsr, appId);
+  }
+
+  @PUT
+  @Path(RMWSConsts.APPS_APPID_QUEUE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response updateAppQueue(AppQueue targetQueue,
+      @Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().updateAppQueue(targetQueue, hsr,
+        appId);
+  }
+
+  @POST
+  @Path(RMWSConsts.APPS_NEW_APPLICATION)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response createNewApplication(@Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().createNewApplication(hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.APPS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response submitApplication(ApplicationSubmissionContextInfo newApp,
+      @Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().submitApplication(newApp, hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.DELEGATION_TOKEN)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response postDelegationToken(DelegationToken tokenData,
+      @Context HttpServletRequest hsr) throws AuthorizationException,
+      IOException, InterruptedException, Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().postDelegationToken(tokenData, hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.DELEGATION_TOKEN_EXPIRATION)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response postDelegationTokenExpiration(@Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().postDelegationTokenExpiration(hsr);
+  }
+
+  @DELETE
+  @Path(RMWSConsts.DELEGATION_TOKEN)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response cancelDelegationToken(@Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().cancelDelegationToken(hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.RESERVATION_NEW)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response createNewReservation(@Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().createNewReservation(hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.RESERVATION_SUBMIT)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response submitReservation(ReservationSubmissionRequestInfo resContext,
+      @Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().submitReservation(resContext, hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.RESERVATION_UPDATE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response updateReservation(ReservationUpdateRequestInfo resContext,
+      @Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().updateReservation(resContext, hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.RESERVATION_DELETE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response deleteReservation(ReservationDeleteRequestInfo resContext,
+      @Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().deleteReservation(resContext, hsr);
+  }
+
+  @GET
+  @Path(RMWSConsts.RESERVATION_LIST)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response listReservation(
+      @QueryParam(RMWSConsts.QUEUE) @DefaultValue(DEFAULT_QUEUE) String queue,
+      @QueryParam(RMWSConsts.RESERVATION_ID) @DefaultValue(DEFAULT_RESERVATION_ID) String reservationId,
+      @QueryParam(RMWSConsts.START_TIME) @DefaultValue(DEFAULT_START_TIME) long startTime,
+      @QueryParam(RMWSConsts.END_TIME) @DefaultValue(DEFAULT_END_TIME) long endTime,
+      @QueryParam(RMWSConsts.INCLUDE_RESOURCE) @DefaultValue(DEFAULT_INCLUDE_RESOURCE) boolean includeResourceAllocations,
+      @Context HttpServletRequest hsr) throws Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().listReservation(queue, reservationId,
+        startTime, endTime, includeResourceAllocations, hsr);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_TIMEOUTS_TYPE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppTimeoutInfo getAppTimeout(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId,
+      @PathParam(RMWSConsts.TYPE) String type) throws AuthorizationException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppTimeout(hsr, appId, type);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_TIMEOUTS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppTimeoutsInfo getAppTimeouts(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppTimeouts(hsr, appId);
+  }
+
+  @PUT
+  @Path(RMWSConsts.APPS_TIMEOUT)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response updateApplicationTimeout(AppTimeoutInfo appTimeout,
+      @Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().updateApplicationTimeout(appTimeout,
+        hsr, appId);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID_APPATTEMPTS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppAttemptsInfo getAppAttempts(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppAttempts(hsr, appId);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID_APPATTEMPTS_APPATTEMPTID)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  public org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo getAppAttempt(
+      @Context HttpServletRequest req, @Context HttpServletResponse res,
+      @PathParam(RMWSConsts.APPID) String appId,
+      @PathParam(RMWSConsts.APPATTEMPTID) String appAttemptId) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppAttempt(req, res, appId,
+        appAttemptId);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID_APPATTEMPTS_APPATTEMPTID_CONTAINERS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  public ContainersInfo getContainers(@Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam(RMWSConsts.APPID) String appId,
+      @PathParam(RMWSConsts.APPATTEMPTID) String appAttemptId) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getContainers(req, res, appId,
+        appAttemptId);
+  }
+
+  @GET
+  @Path(RMWSConsts.GET_CONTAINER)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  public ContainerInfo getContainer(@Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam(RMWSConsts.APPID) String appId,
+      @PathParam(RMWSConsts.APPATTEMPTID) String appAttemptId,
+      @PathParam(RMWSConsts.CONTAINERID) String containerId) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getContainer(req, res, appId,
+        appAttemptId, containerId);
+  }
+
+  @VisibleForTesting
+  protected void setResponse(HttpServletResponse response) {
+    this.response = response;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/package-info.java
new file mode 100644
index 0000000..bd94ead
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+/** Router WebApp package. **/
+package org.apache.hadoop.yarn.server.router.webapp;

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/BaseRouterWebServicesTest.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/BaseRouterWebServicesTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.java
new file mode 100644
index 0000000..223690f
--- /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/BaseRouterWebServicesTest.java
@@ -0,0 +1,601 @@
+/**
+ * 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 static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.Response;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+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.Router;
+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.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.mockito.Mockito;
+
+/**
+ * Base class for all the RouterRMAdminService test cases. It provides utility
+ * methods that can be used by the concrete test case classes.
+ *
+ */
+public abstract class BaseRouterWebServicesTest {
+
+  private YarnConfiguration conf;
+
+  private Router router;
+  public final static int TEST_MAX_CACHE_SIZE = 10;
+
+  private RouterWebServices routerWebService;
+
+  @Before
+  public void setup() {
+    conf = new YarnConfiguration();
+
+    String mockPassThroughInterceptorClass =
+        PassThroughRESTRequestInterceptor.class.getName();
+
+    // Create a request intercepter pipeline for testing. The last one in the
+    // chain will call the mock resource manager. The others in the chain will
+    // simply forward it to the next one in the chain
+    conf.set(YarnConfiguration.ROUTER_WEBAPP_INTERCEPTOR_CLASS_PIPELINE,
+        mockPassThroughInterceptorClass + "," + mockPassThroughInterceptorClass
+            + "," + mockPassThroughInterceptorClass + ","
+            + MockRESTRequestInterceptor.class.getName());
+
+    conf.setInt(YarnConfiguration.ROUTER_PIPELINE_CACHE_MAX_SIZE,
+        TEST_MAX_CACHE_SIZE);
+
+    router = spy(new Router());
+    Mockito.doNothing().when(router).startWepApp();
+    routerWebService = new RouterWebServices(router, conf);
+    routerWebService.setResponse(mock(HttpServletResponse.class));
+
+    router.init(conf);
+    router.start();
+  }
+
+  @After
+  public void tearDown() {
+    if (router != null) {
+      router.stop();
+    }
+  }
+
+  protected RouterWebServices getRouterWebServices() {
+    Assert.assertNotNull(this.routerWebService);
+    return this.routerWebService;
+  }
+
+  protected ClusterInfo get(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ClusterInfo>() {
+          @Override
+          public ClusterInfo run() throws Exception {
+            return routerWebService.get();
+          }
+        });
+  }
+
+  protected ClusterInfo getClusterInfo(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ClusterInfo>() {
+          @Override
+          public ClusterInfo run() throws Exception {
+            return routerWebService.getClusterInfo();
+          }
+        });
+  }
+
+  protected ClusterMetricsInfo getClusterMetricsInfo(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ClusterMetricsInfo>() {
+          @Override
+          public ClusterMetricsInfo run() throws Exception {
+            return routerWebService.getClusterMetricsInfo();
+          }
+        });
+  }
+
+  protected SchedulerTypeInfo getSchedulerInfo(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<SchedulerTypeInfo>() {
+          @Override
+          public SchedulerTypeInfo run() throws Exception {
+            return routerWebService.getSchedulerInfo();
+          }
+        });
+  }
+
+  protected String dumpSchedulerLogs(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<String>() {
+          @Override
+          public String run() throws Exception {
+            return routerWebService.dumpSchedulerLogs(null, null);
+          }
+        });
+  }
+
+  protected NodesInfo getNodes(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<NodesInfo>() {
+          @Override
+          public NodesInfo run() throws Exception {
+            return routerWebService.getNodes(null);
+          }
+        });
+  }
+
+  protected NodeInfo getNode(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<NodeInfo>() {
+          @Override
+          public NodeInfo run() throws Exception {
+            return routerWebService.getNode(null);
+          }
+        });
+  }
+
+  protected AppsInfo getApps(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppsInfo>() {
+          @Override
+          public AppsInfo run() throws Exception {
+            return routerWebService.getApps(null, null, null, null, null, null,
+                null, null, null, null, null, null, null, null);
+          }
+        });
+  }
+
+  protected ActivitiesInfo getActivities(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ActivitiesInfo>() {
+          @Override
+          public ActivitiesInfo run() throws Exception {
+            return routerWebService.getActivities(null, null);
+          }
+        });
+  }
+
+  protected AppActivitiesInfo getAppActivities(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppActivitiesInfo>() {
+          @Override
+          public AppActivitiesInfo run() throws Exception {
+            return routerWebService.getAppActivities(null, null, null);
+          }
+        });
+  }
+
+  protected ApplicationStatisticsInfo getAppStatistics(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ApplicationStatisticsInfo>() {
+          @Override
+          public ApplicationStatisticsInfo run() throws Exception {
+            return routerWebService.getAppStatistics(null, null, null);
+          }
+        });
+  }
+
+  protected AppInfo getApp(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppInfo>() {
+          @Override
+          public AppInfo run() throws Exception {
+            return routerWebService.getApp(null, null, null);
+          }
+        });
+  }
+
+  protected AppState getAppState(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppState>() {
+          @Override
+          public AppState run() throws Exception {
+            return routerWebService.getAppState(null, null);
+          }
+        });
+  }
+
+  protected Response updateAppState(String user) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.updateAppState(null, null, null);
+          }
+        });
+  }
+
+  protected NodeToLabelsInfo getNodeToLabels(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<NodeToLabelsInfo>() {
+          @Override
+          public NodeToLabelsInfo run() throws Exception {
+            return routerWebService.getNodeToLabels(null);
+          }
+        });
+  }
+
+  protected LabelsToNodesInfo getLabelsToNodes(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<LabelsToNodesInfo>() {
+          @Override
+          public LabelsToNodesInfo run() throws Exception {
+            return routerWebService.getLabelsToNodes(null);
+          }
+        });
+  }
+
+  protected Response replaceLabelsOnNodes(String user) throws Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.replaceLabelsOnNodes(null, null);
+          }
+        });
+  }
+
+  protected Response replaceLabelsOnNode(String user) throws Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.replaceLabelsOnNode(null, null, null);
+          }
+        });
+  }
+
+  protected NodeLabelsInfo getClusterNodeLabels(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<NodeLabelsInfo>() {
+          @Override
+          public NodeLabelsInfo run() throws Exception {
+            return routerWebService.getClusterNodeLabels(null);
+          }
+        });
+  }
+
+  protected Response addToClusterNodeLabels(String user) throws Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.addToClusterNodeLabels(null, null);
+          }
+        });
+  }
+
+  protected Response removeFromCluserNodeLabels(String user) throws Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.removeFromCluserNodeLabels(null, null);
+          }
+        });
+  }
+
+  protected NodeLabelsInfo getLabelsOnNode(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<NodeLabelsInfo>() {
+          @Override
+          public NodeLabelsInfo run() throws Exception {
+            return routerWebService.getLabelsOnNode(null, null);
+          }
+        });
+  }
+
+  protected AppPriority getAppPriority(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppPriority>() {
+          @Override
+          public AppPriority run() throws Exception {
+            return routerWebService.getAppPriority(null, null);
+          }
+        });
+  }
+
+  protected Response updateApplicationPriority(String user)
+      throws AuthorizationException, YarnException, InterruptedException,
+      IOException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.updateApplicationPriority(null, null, null);
+          }
+        });
+  }
+
+  protected AppQueue getAppQueue(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppQueue>() {
+          @Override
+          public AppQueue run() throws Exception {
+            return routerWebService.getAppQueue(null, null);
+          }
+        });
+  }
+
+  protected Response updateAppQueue(String user) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.updateAppQueue(null, null, null);
+          }
+        });
+  }
+
+  protected Response createNewApplication(String user)
+      throws AuthorizationException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.createNewApplication(null);
+          }
+        });
+  }
+
+  protected Response submitApplication(String user)
+      throws AuthorizationException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.submitApplication(null, null);
+          }
+        });
+  }
+
+  protected Response postDelegationToken(String user)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.postDelegationToken(null, null);
+          }
+        });
+  }
+
+  protected Response postDelegationTokenExpiration(String user)
+      throws AuthorizationException, IOException, Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.postDelegationTokenExpiration(null);
+          }
+        });
+  }
+
+  protected Response cancelDelegationToken(String user)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.cancelDelegationToken(null);
+          }
+        });
+  }
+
+  protected Response createNewReservation(String user)
+      throws AuthorizationException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.createNewReservation(null);
+          }
+        });
+  }
+
+  protected Response submitReservation(String user)
+      throws AuthorizationException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.submitReservation(null, null);
+          }
+        });
+  }
+
+  protected Response updateReservation(String user)
+      throws AuthorizationException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.updateReservation(null, null);
+          }
+        });
+  }
+
+  protected Response deleteReservation(String user)
+      throws AuthorizationException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.deleteReservation(null, null);
+          }
+        });
+  }
+
+  protected Response listReservation(String user) throws Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.listReservation(null, null, 0, 0, false,
+                null);
+          }
+        });
+  }
+
+  protected AppTimeoutInfo getAppTimeout(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppTimeoutInfo>() {
+          @Override
+          public AppTimeoutInfo run() throws Exception {
+            return routerWebService.getAppTimeout(null, null, null);
+          }
+        });
+  }
+
+  protected AppTimeoutsInfo getAppTimeouts(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppTimeoutsInfo>() {
+          @Override
+          public AppTimeoutsInfo run() throws Exception {
+            return routerWebService.getAppTimeouts(null, null);
+          }
+        });
+  }
+
+  protected Response updateApplicationTimeout(String user)
+      throws AuthorizationException, YarnException, InterruptedException,
+      IOException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.updateApplicationTimeout(null, null, null);
+          }
+        });
+  }
+
+  protected AppAttemptsInfo getAppAttempts(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppAttemptsInfo>() {
+          @Override
+          public AppAttemptsInfo run() throws Exception {
+            return routerWebService.getAppAttempts(null, null);
+          }
+        });
+  }
+
+  protected AppAttemptInfo getAppAttempt(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppAttemptInfo>() {
+          @Override
+          public AppAttemptInfo run() throws Exception {
+            return routerWebService.getAppAttempt(null, null, null, null);
+          }
+        });
+  }
+
+  protected ContainersInfo getContainers(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ContainersInfo>() {
+          @Override
+          public ContainersInfo run() throws Exception {
+            return routerWebService.getContainers(null, null, null, null);
+          }
+        });
+  }
+
+  protected ContainerInfo getContainer(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ContainerInfo>() {
+          @Override
+          public ContainerInfo run() throws Exception {
+            return routerWebService.getContainer(null, null, null, null, null);
+          }
+        });
+  }
+
+  protected RequestInterceptorChainWrapper getInterceptorChain(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<RequestInterceptorChainWrapper>() {
+          @Override
+          public RequestInterceptorChainWrapper run() throws Exception {
+            return routerWebService.getInterceptorChain();
+          }
+        });
+  }
+
+}

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/JavaProcess.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/JavaProcess.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/JavaProcess.java
new file mode 100644
index 0000000..d32013f
--- /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/JavaProcess.java
@@ -0,0 +1,52 @@
+/**
+* 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.File;
+import java.io.IOException;
+
+/**
+ * Helper class to start a new process.
+ */
+public class JavaProcess {
+
+  private Process process = null;
+
+  public JavaProcess(Class<?> klass) throws IOException, InterruptedException {
+    String javaHome = System.getProperty("java.home");
+    String javaBin =
+        javaHome + File.separator + "bin" + File.separator + "java";
+    String classpath = System.getProperty("java.class.path");
+    classpath = classpath.concat("./src/test/resources");
+    String className = klass.getCanonicalName();
+    ProcessBuilder builder =
+        new ProcessBuilder(javaBin, "-cp", classpath, className);
+    builder.inheritIO();
+    process = builder.start();
+  }
+
+  public void stop() throws InterruptedException {
+    if (process != null) {
+      process.destroy();
+      process.waitFor();
+      process.exitValue();
+    }
+  }
+
+}
\ No newline at end of file

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/MockRESTRequestInterceptor.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/MockRESTRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockRESTRequestInterceptor.java
new file mode 100644
index 0000000..69afdea
--- /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/MockRESTRequestInterceptor.java
@@ -0,0 +1,340 @@
+/**
+ * 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 javax.ws.rs.core.Response.Status;
+
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+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;
+
+/**
+ * This class mocks the RESTRequestInterceptor.
+ */
+public class MockRESTRequestInterceptor extends AbstractRESTRequestInterceptor {
+
+  @Override
+  public void setNextInterceptor(RESTRequestInterceptor next) {
+    throw new YarnRuntimeException(
+        "setNextInterceptor is being called on MockRESTRequestInterceptor,"
+            + "which should be the last one in the chain. "
+            + "Check if the interceptor pipeline configuration is correct");
+  }
+
+  @Override
+  public ClusterInfo get() {
+    return new ClusterInfo();
+  }
+
+  @Override
+  public ClusterInfo getClusterInfo() {
+    return new ClusterInfo();
+  }
+
+  @Override
+  public ClusterMetricsInfo getClusterMetricsInfo() {
+    return new ClusterMetricsInfo();
+  }
+
+  @Override
+  public SchedulerTypeInfo getSchedulerInfo() {
+    return new SchedulerTypeInfo();
+  }
+
+  @Override
+  public String dumpSchedulerLogs(String time, HttpServletRequest hsr)
+      throws IOException {
+    return "Done";
+  }
+
+  @Override
+  public NodesInfo getNodes(String states) {
+    return new NodesInfo();
+  }
+
+  @Override
+  public NodeInfo getNode(String nodeId) {
+    return new NodeInfo();
+  }
+
+  @SuppressWarnings("checkstyle:parameternumber")
+  @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 new AppsInfo();
+  }
+
+  @Override
+  public ActivitiesInfo getActivities(HttpServletRequest hsr, String nodeId) {
+    return new ActivitiesInfo();
+  }
+
+  @Override
+  public AppActivitiesInfo getAppActivities(HttpServletRequest hsr,
+      String appId, String time) {
+    return new AppActivitiesInfo();
+  }
+
+  @Override
+  public ApplicationStatisticsInfo getAppStatistics(HttpServletRequest hsr,
+      Set<String> stateQueries, Set<String> typeQueries) {
+    return new ApplicationStatisticsInfo();
+  }
+
+  @Override
+  public AppInfo getApp(HttpServletRequest hsr, String appId,
+      Set<String> unselectedFields) {
+    return new AppInfo();
+  }
+
+  @Override
+  public AppState getAppState(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return new AppState();
+  }
+
+  @Override
+  public Response updateAppState(AppState targetState, HttpServletRequest hsr,
+      String appId) throws AuthorizationException, YarnException,
+      InterruptedException, IOException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public NodeToLabelsInfo getNodeToLabels(HttpServletRequest hsr)
+      throws IOException {
+    return new NodeToLabelsInfo();
+  }
+
+  @Override
+  public LabelsToNodesInfo getLabelsToNodes(Set<String> labels)
+      throws IOException {
+    return new LabelsToNodesInfo();
+  }
+
+  @Override
+  public Response replaceLabelsOnNodes(NodeToLabelsEntryList newNodeToLabels,
+      HttpServletRequest hsr) throws Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response replaceLabelsOnNode(Set<String> newNodeLabelsName,
+      HttpServletRequest hsr, String nodeId) throws Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public NodeLabelsInfo getClusterNodeLabels(HttpServletRequest hsr)
+      throws IOException {
+    return new NodeLabelsInfo();
+  }
+
+  @Override
+  public Response addToClusterNodeLabels(NodeLabelsInfo newNodeLabels,
+      HttpServletRequest hsr) throws Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response removeFromCluserNodeLabels(Set<String> oldNodeLabels,
+      HttpServletRequest hsr) throws Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public NodeLabelsInfo getLabelsOnNode(HttpServletRequest hsr, String nodeId)
+      throws IOException {
+    return new NodeLabelsInfo();
+  }
+
+  @Override
+  public AppPriority getAppPriority(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return new AppPriority();
+  }
+
+  @Override
+  public Response updateApplicationPriority(AppPriority targetPriority,
+      HttpServletRequest hsr, String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public AppQueue getAppQueue(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return new AppQueue();
+  }
+
+  @Override
+  public Response updateAppQueue(AppQueue targetQueue, HttpServletRequest hsr,
+      String appId) throws AuthorizationException, YarnException,
+      InterruptedException, IOException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response createNewApplication(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response submitApplication(ApplicationSubmissionContextInfo newApp,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response postDelegationToken(DelegationToken tokenData,
+      HttpServletRequest hsr) throws AuthorizationException, IOException,
+      InterruptedException, Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response postDelegationTokenExpiration(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response cancelDelegationToken(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response createNewReservation(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response submitReservation(ReservationSubmissionRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response updateReservation(ReservationUpdateRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response deleteReservation(ReservationDeleteRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response listReservation(String queue, String reservationId,
+      long startTime, long endTime, boolean includeResourceAllocations,
+      HttpServletRequest hsr) throws Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public AppTimeoutInfo getAppTimeout(HttpServletRequest hsr, String appId,
+      String type) throws AuthorizationException {
+    return new AppTimeoutInfo();
+  }
+
+  @Override
+  public AppTimeoutsInfo getAppTimeouts(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return new AppTimeoutsInfo();
+  }
+
+  @Override
+  public Response updateApplicationTimeout(AppTimeoutInfo appTimeout,
+      HttpServletRequest hsr, String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public AppAttemptsInfo getAppAttempts(HttpServletRequest hsr, String appId) {
+    return new AppAttemptsInfo();
+  }
+
+  @Override
+  public AppAttemptInfo getAppAttempt(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId) {
+    return new AppAttemptInfo();
+  }
+
+  @Override
+  public ContainersInfo getContainers(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId) {
+    return new ContainersInfo();
+  }
+
+  @Override
+  public ContainerInfo getContainer(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId,
+      String containerId) {
+    return new ContainerInfo();
+  }
+
+}
\ 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