diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index 26b8041..7def720 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -2664,6 +2664,30 @@ public static boolean isAclEnabled(Configuration conf) { ROUTER_PREFIX + "submit.retry"; public static final int DEFAULT_ROUTER_CLIENTRM_SUBMIT_RETRY = 3; + public static final String ROUTER_WEBAPP_PREFIX = ROUTER_PREFIX + "webapp."; + + /** The address of the Router web application. */ + public static final String ROUTER_WEBAPP_ADDRESS = + ROUTER_WEBAPP_PREFIX + " address"; + + public static final int DEFAULT_ROUTER_WEBAPP_PORT = 8089; + public static final String DEFAULT_ROUTER_WEBAPP_ADDRESS = + "0.0.0.0:" + DEFAULT_ROUTER_WEBAPP_PORT; + + /** The https address of the Router web application. */ + public static final String ROUTER_WEBAPP_HTTPS_ADDRESS = + ROUTER_WEBAPP_PREFIX + "https.address"; + + public static final int DEFAULT_ROUTER_WEBAPP_HTTPS_PORT = 8091; + public static final String DEFAULT_ROUTER_WEBAPP_HTTPS_ADDRESS = + "0.0.0.0:" + DEFAULT_ROUTER_WEBAPP_HTTPS_PORT; + + public static final String ROUTER_WEBAPP_INTERCEPTOR_CLASS_PIPELINE = + ROUTER_WEBAPP_PREFIX + "interceptor-class.pipeline"; + public static final String DEFAULT_ROUTER_WEBAPP_INTERCEPTOR_CLASS = + "org.apache.hadoop.yarn.server.router.webapp." + + "DefaultRequestInterceptorREST"; + //////////////////////////////// // Other Configs //////////////////////////////// diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java index eabd9b3..fb0a9bf 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java @@ -129,6 +129,20 @@ public static String getRMWebAppURLWithoutScheme(Configuration conf) { return getRMWebAppURLWithoutScheme(conf, false); } + public static String getRouterWebAppURLWithScheme(Configuration conf) { + return getHttpSchemePrefix(conf) + getRouterWebAppURLWithoutScheme(conf); + } + + public static String getRouterWebAppURLWithoutScheme(Configuration conf) { + if (YarnConfiguration.useHttps(conf)) { + return conf.get(YarnConfiguration.ROUTER_WEBAPP_HTTPS_ADDRESS, + YarnConfiguration.DEFAULT_ROUTER_WEBAPP_HTTPS_ADDRESS); + } else { + return conf.get(YarnConfiguration.ROUTER_WEBAPP_ADDRESS, + YarnConfiguration.DEFAULT_ROUTER_WEBAPP_ADDRESS); + } + } + public static List getProxyHostsAndPortsForAmFilter( Configuration conf) { List addrs = new ArrayList(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeIDsInfo.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeIDsInfo.java index c23b02a..9509d60 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeIDsInfo.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeIDsInfo.java @@ -26,7 +26,7 @@ import javax.xml.bind.annotation.XmlElement; import javax.xml.bind.annotation.XmlRootElement; -@XmlRootElement(name = "labelsToNodesInfo") +@XmlRootElement(name = "nodeIdsInfo") @XmlAccessorType(XmlAccessType.FIELD) public class NodeIDsInfo { diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWSConsts.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWSConsts.java index 23d4bb1..a5066ec 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWSConsts.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWSConsts.java @@ -168,6 +168,12 @@ */ public static final String APPS_TIMEOUT = "/apps/{appid}/timeout"; + /** + * Path for {@code RouterWebServices#getContainer}. + */ + public static final String GET_CONTAINER = + "/apps/{appid}/appattempts/{appattemptid}/containers/{containerid}"; + // ----------------QueryParams for RMWebServiceProtocol---------------- public static final String TIME = "time"; @@ -194,6 +200,13 @@ public static final String END_TIME = "end-time"; public static final String INCLUDE_RESOURCE = "include-resource-allocations"; public static final String TYPE = "type"; + public static final String CONTAINERID = "containerid"; + public static final String APPATTEMPTS = "appattempts"; + public static final String TIMEOUTS = "timeouts"; + public static final String PRIORITY = "priority"; + public static final String TIMEOUT = "timeout"; + public static final String ATTEMPTS = "appattempts"; + public static final String GET_LABELS = "get-labels"; private RMWSConsts() { // not called diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppUtil.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppUtil.java index 263828b..ea1d810 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppUtil.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppUtil.java @@ -18,11 +18,15 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp; +import java.security.Principal; import java.util.ArrayList; +import javax.servlet.http.HttpServletRequest; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.http.lib.StaticUserWebFilter; import org.apache.hadoop.security.AuthenticationFilterInitializer; import org.apache.hadoop.security.HttpCrossOriginFilterInitializer; @@ -146,4 +150,45 @@ public static void setupSecurityAndFilters(Configuration conf, } } } + + /** + * Helper method to retrieve the UserGroupInformation from the + * HttpServletRequest. + * + * @param hsr the servlet request + * @param usePrincipal true if we need to use the principal user, remote + * otherwise. + * @return the user group information of the caller. + **/ + public static UserGroupInformation getCallerUserGroupInformation( + HttpServletRequest hsr, boolean usePrincipal) { + + String remoteUser = hsr.getRemoteUser(); + if (usePrincipal) { + Principal princ = hsr.getUserPrincipal(); + remoteUser = princ == null ? null : princ.getName(); + } + + UserGroupInformation callerUGI = null; + if (remoteUser != null) { + callerUGI = UserGroupInformation.createRemoteUser(remoteUser); + } + + return callerUGI; + } + + /** + * Helper method to retrieve if the UserGroupInformation is a static one. + * + * @param callerUGI the UserGroupInformation we want to analyze. + * @param conf RM configuration. + * @return true if the UGI is static, false otherwise. + **/ + public static boolean isStaticUser(UserGroupInformation callerUGI, + Configuration conf) { + String staticUser = + conf.get(CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER, + CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER); + return staticUser.equals(callerUGI.getUserName()); + } } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java index acfb2b8..80d4839 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java @@ -22,7 +22,6 @@ import java.lang.reflect.UndeclaredThrowableException; import java.nio.ByteBuffer; import java.security.AccessControlException; -import java.security.Principal; import java.security.PrivilegedExceptionAction; import java.text.ParseException; import java.util.ArrayList; @@ -61,7 +60,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.http.JettyUtils; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.Text; @@ -247,7 +245,8 @@ public RMWebServices(final ResourceManager rm, Configuration conf) { protected Boolean hasAccess(RMApp app, HttpServletRequest hsr) { // Check for the authorization. - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); List forwardedAddresses = null; String forwardedFor = hsr.getHeader(RMWSConsts.FORWARDED_FOR); if (forwardedFor != null) { @@ -330,7 +329,8 @@ public SchedulerTypeInfo getSchedulerInfo() { public String dumpSchedulerLogs(@FormParam(RMWSConsts.TIME) String time, @Context HttpServletRequest hsr) throws IOException { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); ApplicationACLsManager aclsManager = rm.getApplicationACLsManager(); if (aclsManager.areACLsEnabled()) { if (callerUGI == null || !aclsManager.isAdmin(callerUGI)) { @@ -903,14 +903,15 @@ public ContainerInfo getContainer(@Context HttpServletRequest req, } @GET - @Path("/apps/{appid}/state") + @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(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); String userName = ""; if (callerUGI != null) { userName = callerUGI.getUserName(); @@ -947,13 +948,15 @@ public Response updateAppState(AppState targetState, YarnException, InterruptedException, IOException { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { String msg = "Unable to obtain user name, user not authenticated"; throw new AuthorizationException(msg); } - if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) { + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, conf)) { String msg = "The default static user cannot carry out this operation."; return Response.status(Status.FORBIDDEN).entity(msg).build(); } @@ -1067,9 +1070,9 @@ public Response replaceLabelsOnNodes( MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 }) @Override public Response replaceLabelsOnNode( - @QueryParam("labels") Set newNodeLabelsName, - @Context HttpServletRequest hsr, @PathParam("nodeId") String nodeId) - throws Exception { + @QueryParam(RMWSConsts.LABELS) Set newNodeLabelsName, + @Context HttpServletRequest hsr, + @PathParam(RMWSConsts.NODEID) String nodeId) throws Exception { NodeId nid = ConverterUtils.toNodeIdWithDefaultPort(nodeId); Map> newLabelsForNode = new HashMap>(); @@ -1087,7 +1090,8 @@ private Response replaceLabelsOnNode( NodeLabelsUtils.verifyCentralizedNodeLabelConfEnabled("replaceLabelsOnNode", isCentralizedNodeLabelConfiguration); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { String msg = "Unable to obtain user name, user not authenticated for" + " post to ..." + operation; @@ -1134,7 +1138,8 @@ public Response addToClusterNodeLabels(final NodeLabelsInfo newNodeLabels, @Context HttpServletRequest hsr) throws Exception { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { String msg = "Unable to obtain user name, user not authenticated for" + " post to .../add-node-labels"; @@ -1167,7 +1172,8 @@ public Response removeFromCluserNodeLabels( @Context HttpServletRequest hsr) throws Exception { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { String msg = "Unable to obtain user name, user not authenticated for" + " post to .../remove-node-labels"; @@ -1267,7 +1273,8 @@ public KillApplicationResponse run() public AppPriority getAppPriority(@Context HttpServletRequest hsr, @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); String userName = "UNKNOWN-USER"; if (callerUGI != null) { userName = callerUGI.getUserName(); @@ -1303,13 +1310,15 @@ public Response updateApplicationPriority(AppPriority targetPriority, throw new YarnException("Target Priority cannot be null"); } - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { throw new AuthorizationException( "Unable to obtain user name, user not authenticated"); } - if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) { + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, conf)) { return Response.status(Status.FORBIDDEN) .entity("The default static user cannot carry out this operation.") .build(); @@ -1384,7 +1393,8 @@ public Void run() throws IOException, YarnException { public AppQueue getAppQueue(@Context HttpServletRequest hsr, @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); String userName = "UNKNOWN-USER"; if (callerUGI != null) { userName = callerUGI.getUserName(); @@ -1417,13 +1427,15 @@ public Response updateAppQueue(AppQueue targetQueue, YarnException, InterruptedException, IOException { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { String msg = "Unable to obtain user name, user not authenticated"; throw new AuthorizationException(msg); } - if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) { + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, conf)) { String msg = "The default static user cannot carry out this operation."; return Response.status(Status.FORBIDDEN).entity(msg).build(); } @@ -1506,30 +1518,6 @@ private RMApp getRMAppForAppId(String appId) { return app; } - private UserGroupInformation getCallerUserGroupInformation( - HttpServletRequest hsr, boolean usePrincipal) { - - String remoteUser = hsr.getRemoteUser(); - if (usePrincipal) { - Principal princ = hsr.getUserPrincipal(); - remoteUser = princ == null ? null : princ.getName(); - } - - UserGroupInformation callerUGI = null; - if (remoteUser != null) { - callerUGI = UserGroupInformation.createRemoteUser(remoteUser); - } - - return callerUGI; - } - - private boolean isStaticUser(UserGroupInformation callerUGI) { - String staticUser = - conf.get(CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER, - CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER); - return staticUser.equals(callerUGI.getUserName()); - } - @POST @Path(RMWSConsts.APPS_NEW_APPLICATION) @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8, @@ -1538,12 +1526,14 @@ private boolean isStaticUser(UserGroupInformation callerUGI) { public Response createNewApplication(@Context HttpServletRequest hsr) throws AuthorizationException, IOException, InterruptedException { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { throw new AuthorizationException( "Unable to obtain user name, " + "user not authenticated"); } - if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) { + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, conf)) { String msg = "The default static user cannot carry out this operation."; return Response.status(Status.FORBIDDEN).entity(msg).build(); } @@ -1567,13 +1557,15 @@ public Response submitApplication(ApplicationSubmissionContextInfo newApp, throws AuthorizationException, IOException, InterruptedException { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { throw new AuthorizationException( "Unable to obtain user name, " + "user not authenticated"); } - if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) { + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, conf)) { String msg = "The default static user cannot carry out this operation."; return Response.status(Status.FORBIDDEN).entity(msg).build(); } @@ -1780,7 +1772,8 @@ private Credentials createCredentials(CredentialsInfo credentials) { private UserGroupInformation createKerberosUserGroupInformation( HttpServletRequest hsr) throws AuthorizationException, YarnException { - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { String msg = "Unable to obtain user name, user not authenticated"; throw new AuthorizationException(msg); @@ -2039,12 +2032,14 @@ public CancelDelegationTokenResponse run() public Response createNewReservation(@Context HttpServletRequest hsr) throws AuthorizationException, IOException, InterruptedException { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { throw new AuthorizationException( "Unable to obtain user name, " + "user not authenticated"); } - if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) { + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, conf)) { String msg = "The default static user cannot carry out this operation."; return Response.status(Status.FORBIDDEN).entity(msg).build(); } @@ -2088,12 +2083,14 @@ public Response submitReservation(ReservationSubmissionRequestInfo resContext, throws AuthorizationException, IOException, InterruptedException { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { throw new AuthorizationException( "Unable to obtain user name, " + "user not authenticated"); } - if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) { + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, conf)) { String msg = "The default static user cannot carry out this operation."; return Response.status(Status.FORBIDDEN).entity(msg).build(); } @@ -2185,12 +2182,14 @@ public Response updateReservation(ReservationUpdateRequestInfo resContext, throws AuthorizationException, IOException, InterruptedException { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { throw new AuthorizationException( "Unable to obtain user name, " + "user not authenticated"); } - if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) { + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, conf)) { String msg = "The default static user cannot carry out this operation."; return Response.status(Status.FORBIDDEN).entity(msg).build(); } @@ -2283,12 +2282,14 @@ public Response deleteReservation(ReservationDeleteRequestInfo resContext, throws AuthorizationException, IOException, InterruptedException { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { throw new AuthorizationException( "Unable to obtain user name, " + "user not authenticated"); } - if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) { + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, conf)) { String msg = "The default static user cannot carry out this operation."; return Response.status(Status.FORBIDDEN).entity(msg).build(); } @@ -2344,12 +2345,14 @@ public Response listReservation( final ReservationListRequest request = ReservationListRequest.newInstance( queue, reservationId, startTime, endTime, includeResourceAllocations); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { throw new AuthorizationException( "Unable to obtain user name, " + "user not authenticated"); } - if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) { + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, conf)) { String msg = "The default static user cannot carry out this operation."; return Response.status(Status.FORBIDDEN).entity(msg).build(); } @@ -2397,13 +2400,15 @@ public AppTimeoutInfo getAppTimeout(@Context HttpServletRequest hsr, private RMApp validateAppTimeoutRequest(HttpServletRequest hsr, String appId) { - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); String userName = "UNKNOWN-USER"; if (callerUGI != null) { userName = callerUGI.getUserName(); } - if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) { + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, conf)) { String msg = "The default static user cannot carry out this operation."; RMAuditLogger.logFailure(userName, AuditConstants.GET_APP_TIMEOUTS, "UNKNOWN", "RMWebService", msg); @@ -2489,13 +2494,15 @@ public Response updateApplicationTimeout(AppTimeoutInfo appTimeout, YarnException, InterruptedException, IOException { init(); - UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true); + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); if (callerUGI == null) { throw new AuthorizationException( "Unable to obtain user name, user not authenticated"); } - if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) { + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, conf)) { return Response.status(Status.FORBIDDEN) .entity("The default static user cannot carry out this operation.") .build(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LabelsToNodesInfo.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LabelsToNodesInfo.java index 41dd410..7de3b64 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LabelsToNodesInfo.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LabelsToNodesInfo.java @@ -40,4 +40,8 @@ public LabelsToNodesInfo() { public Map getLabelsToNodes() { return labelsToNodes; } + + public LabelsToNodesInfo(Map labelsToNodes) { + this.labelsToNodes = labelsToNodes; + } } \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml index f27b2b2..f1d914c 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml @@ -50,13 +50,6 @@ org.apache.hadoop - hadoop-common - test-jar - test - - - - org.apache.hadoop hadoop-yarn-server-common @@ -72,6 +65,28 @@ junit test + + + org.apache.hadoop + hadoop-yarn-server-resourcemanager + + + + org.apache.hadoop + hadoop-yarn-server-nodemanager + + + + org.mockito + mockito-all + test + + + + com.google.inject + guice + + diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java index d2eee5a..5bbaf6f 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.util.ShutdownHookManager; @@ -28,11 +29,19 @@ import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebAppUtil; import org.apache.hadoop.yarn.server.router.clientrm.RouterClientRMService; import org.apache.hadoop.yarn.server.router.rmadmin.RouterRMAdminService; +import org.apache.hadoop.yarn.server.router.webapp.RouterWebApp; +import org.apache.hadoop.yarn.webapp.WebApp; +import org.apache.hadoop.yarn.webapp.WebApps; +import org.apache.hadoop.yarn.webapp.WebApps.Builder; +import org.apache.hadoop.yarn.webapp.util.WebAppUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.annotations.VisibleForTesting; + /** * The router is a stateless YARN component which is the entry point to the * cluster. It can be deployed on multiple nodes behind a Virtual IP (VIP) with @@ -56,6 +65,9 @@ private AtomicBoolean isStopping = new AtomicBoolean(false); private RouterClientRMService clientRMProxyService; private RouterRMAdminService rmAdminProxyService; + private WebApp webApp; + @VisibleForTesting + protected String webAppAddress; /** * Priority of the Router shutdown hook. @@ -79,6 +91,10 @@ protected void serviceInit(Configuration config) throws Exception { // RMAdmin Proxy rmAdminProxyService = createRMAdminProxyService(); addService(rmAdminProxyService); + // WebService + webAppAddress = WebAppUtils.getWebAppBindURL(this.conf, + YarnConfiguration.ROUTER_BIND_HOST, + WebAppUtils.getRouterWebAppURLWithoutScheme(this.conf)); super.serviceInit(conf); } @@ -89,11 +105,15 @@ protected void serviceStart() throws Exception { } catch (IOException e) { throw new YarnRuntimeException("Failed Router login", e); } + startWepApp(); super.serviceStart(); } @Override protected void serviceStop() throws Exception { + if (webApp != null) { + webApp.stop(); + } if (isStopping.getAndSet(true)) { return; } @@ -117,6 +137,23 @@ protected RouterRMAdminService createRMAdminProxyService() { return new RouterRMAdminService(); } + @Private + public WebApp getWebapp() { + return this.webApp; + } + + @VisibleForTesting + public void startWepApp() { + + Configuration conf = getConfig(); + + RMWebAppUtil.setupSecurityAndFilters(conf, null); + + Builder builder = + WebApps.$for("cluster", null, null, "ws").with(conf).at(webAppAddress); + webApp = builder.start(new RouterWebApp(this)); + } + public static void main(String[] argv) { Configuration conf = new YarnConfiguration(); Thread diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/AbstractRESTRequestInterceptor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/AbstractRESTRequestInterceptor.java new file mode 100644 index 0000000..1765370 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/AbstractRESTRequestInterceptor.java @@ -0,0 +1,89 @@ +/** + * 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 org.apache.hadoop.conf.Configuration; + +/** + * Extends the RequestInterceptor class and provides common functionality which + * can can be used and/or extended by other concrete intercepter classes. + */ +public abstract class AbstractRESTRequestInterceptor + implements RESTRequestInterceptor { + + private Configuration conf; + private RESTRequestInterceptor nextInterceptor; + + /** + * Sets the {@link RequestInterceptor} in the chain. + */ + @Override + public void setNextInterceptor(RESTRequestInterceptor nextInterceptor) { + this.nextInterceptor = nextInterceptor; + } + + /** + * Sets the {@link Configuration}. + */ + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + if (this.nextInterceptor != null) { + this.nextInterceptor.setConf(conf); + } + } + + /** + * Gets the {@link Configuration}. + */ + @Override + public Configuration getConf() { + return this.conf; + } + + /** + * Initializes the {@link RESTRequestInterceptor}. + */ + @Override + public void init(String user) { + if (this.nextInterceptor != null) { + this.nextInterceptor.init(user); + } + } + + /** + * Disposes the {@link RESTRequestInterceptor}. + */ + @Override + public void shutdown() { + if (this.nextInterceptor != null) { + this.nextInterceptor.shutdown(); + } + } + + /** + * Gets the next {@link RESTRequestInterceptor} in the chain. + */ + @Override + public RESTRequestInterceptor getNextInterceptor() { + return this.nextInterceptor; + } + +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.java new file mode 100644 index 0000000..af4b6a1 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.java @@ -0,0 +1,1326 @@ +/** + * 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.HashMap; +import java.util.Map; +import java.util.Set; + +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.core.Response; + +import org.apache.commons.lang.NotImplementedException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.security.UserGroupInformation; +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.RMWSConsts; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebAppUtil; +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; +import org.apache.hadoop.yarn.webapp.util.WebAppUtils; + +import com.sun.jersey.api.client.ClientResponse; + +/** + * Extends the AbstractRequestInterceptorClient class and provides an + * implementation that simply forwards the client requests to the resource + * manager. + */ +public final class DefaultRequestInterceptorREST + extends AbstractRESTRequestInterceptor { + + private String webAppAddress; + private static final int STATUS_OK = 200; + + private static final Log LOG = + LogFactory.getLog(DefaultRequestInterceptorREST.class.getName()); + + @Override + public void init(String user) { + webAppAddress = WebAppUtils.getRMWebAppURLWithScheme(getConf()); + } + + @Override + public ClusterInfo get() { + return getClusterInfo(); + } + + @Override + public ClusterInfo getClusterInfo() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.INFO, + HTTPMethods.GET, null, null, null); + // YARN RM can answer Status.OK or null + if (response.getStatus() == STATUS_OK) { + return response.getEntity(ClusterInfo.class); + } + return null; + } + + @Override + public ClusterMetricsInfo getClusterMetricsInfo() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.METRICS, + HTTPMethods.GET, null, null, null); + // YARN RM can answer Status.OK or null + if (response.getStatus() == STATUS_OK) { + return response.getEntity(ClusterMetricsInfo.class); + } + return null; + } + + @Override + public SchedulerTypeInfo getSchedulerInfo() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER, + HTTPMethods.GET, null, null, null); + // YARN RM can answer Status.OK or it throws an exception + if (response.getStatus() == STATUS_OK) { + return response.getEntity(SchedulerTypeInfo.class); + } + RouterWebServiceUtil.retrieveException(response); + return null; + } + + @Override + public String dumpSchedulerLogs(String time, HttpServletRequest hsr) + throws IOException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, getConf())) { + String msg = "The default static user cannot carry out this operation."; + return msg; + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public String run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_LOGS, + HTTPMethods.POST, null, hsr.getParameterMap(), null); + // YARN RM can answer with Status.OK or it throws an exception + if (response.getStatus() == STATUS_OK) { + return response.getEntity(String.class); + } + RouterWebServiceUtil.retrieveException(response); + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public NodesInfo getNodes(String states) { + Map queryParams = null; + if (states != null) { + queryParams = new HashMap<>(); + queryParams.put(RMWSConsts.STATES, new String[] { states }); + } + + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES, + HTTPMethods.GET, null, queryParams, null); + + // YARN RM can answer with Status.OK or it throws an exception + if (response.getStatus() == STATUS_OK) { + return response.getEntity(NodesInfo.class); + } + RouterWebServiceUtil.retrieveException(response); + return null; + } + + @Override + public NodeInfo getNode(String nodeId) { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES, + HTTPMethods.GET, nodeId, null, null); + + // YARN RM can answer with Status.OK or it throws an exception + if (response.getStatus() == STATUS_OK) { + return response.getEntity(NodeInfo.class); + } + RouterWebServiceUtil.retrieveException(response); + return null; + } + + @Override + public AppsInfo getApps(HttpServletRequest hsr, String stateQuery, + Set statesQuery, String finalStatusQuery, String userQuery, + String queueQuery, String count, String startedBegin, String startedEnd, + String finishBegin, String finishEnd, Set applicationTypes, + Set applicationTags) { + + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + LOG.error("Unable to obtain user name, user not authenticated"); + return null; + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public AppsInfo run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, + HTTPMethods.GET, hsr.getPathInfo(), hsr.getParameterMap(), null); + // YARN RM can answer with Status.OK or it throws an exception + if (response.getStatus() == STATUS_OK) { + return response.getEntity(AppsInfo.class); + } + RouterWebServiceUtil.retrieveException(response); + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public ActivitiesInfo getActivities(HttpServletRequest hsr, String nodeId) { + + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + LOG.error("Unable to obtain user name, user not authenticated"); + return null; + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public ActivitiesInfo run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + + RMWSConsts.SCHEDULER_ACTIVITIES, + HTTPMethods.GET, null, hsr.getParameterMap(), null); + // YARN RM can answer with Status.OK or null + if (response.getStatus() == STATUS_OK) { + return response.getEntity(ActivitiesInfo.class); + } + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public AppActivitiesInfo getAppActivities(HttpServletRequest hsr, + String appId, String time) { + + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + LOG.error("Unable to obtain user name, user not authenticated"); + return null; + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public AppActivitiesInfo run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + + RMWSConsts.SCHEDULER_APP_ACTIVITIES, + HTTPMethods.GET, null, hsr.getParameterMap(), null); + // YARN RM can answer with Status.OK or null + if (response.getStatus() == STATUS_OK) { + return response.getEntity(AppActivitiesInfo.class); + } + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public ApplicationStatisticsInfo getAppStatistics(HttpServletRequest hsr, + Set stateQueries, Set typeQueries) { + + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + LOG.error("Unable to obtain user name, user not authenticated"); + return null; + } + + try { + return callerUGI + .doAs(new PrivilegedExceptionAction() { + @Override + public ApplicationStatisticsInfo run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + + RMWSConsts.APP_STATISTICS, + HTTPMethods.GET, null, hsr.getParameterMap(), null); + // YARN RM can answer with Status.OK or it throws an exception + if (response.getStatus() == STATUS_OK) { + return response.getEntity(ApplicationStatisticsInfo.class); + } + RouterWebServiceUtil.retrieveException(response); + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public AppInfo getApp(HttpServletRequest hsr, String appId) { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public AppInfo run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, + HTTPMethods.GET, appId, null, null); + // YARN RM can answer with Status.OK or it throws an exception + if (response.getStatus() == STATUS_OK) { + return response.getEntity(AppInfo.class); + } + RouterWebServiceUtil.retrieveException(response); + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public AppState getAppState(HttpServletRequest hsr, String appId) + throws AuthorizationException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public AppState run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, + HTTPMethods.GET, appId + "/" + RMWSConsts.STATE, null, null); + // YARN RM can answer with Status.OK or it throws an exception + if (response.getStatus() == STATUS_OK) { + return response.getEntity(AppState.class); + } + RouterWebServiceUtil.retrieveException(response); + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public Response updateAppState(AppState targetState, HttpServletRequest hsr, + String appId) throws AuthorizationException, YarnException, + InterruptedException, IOException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, getConf())) { + throw new AuthorizationException( + "The default static user cannot carry out this operation."); + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, + HTTPMethods.PUT, appId + "/" + RMWSConsts.STATE, + hsr.getParameterMap(), targetState); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public NodeToLabelsInfo getNodeToLabels(HttpServletRequest hsr) + throws IOException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public NodeToLabelsInfo run() { + + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + + RMWSConsts.GET_NODE_TO_LABELS, + HTTPMethods.GET, null, null, null); + // YARN RM can answer with Status.OK + if (response.getStatus() == STATUS_OK) { + return response.getEntity(NodeToLabelsInfo.class); + } + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public LabelsToNodesInfo getLabelsToNodes(Set labels) + throws IOException { + Map paramMap = new HashMap<>(); + paramMap.put(RMWSConsts.LABELS, labels.toArray(new String[labels.size()])); + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.LABEL_MAPPINGS, + HTTPMethods.GET, null, paramMap, null); + // YARN RM can answer with Status.OK + if (response.getStatus() == STATUS_OK) { + return response.getEntity(LabelsToNodesInfo.class); + } + return null; + } + + @Override + public Response replaceLabelsOnNodes(NodeToLabelsEntryList newNodeToLabels, + HttpServletRequest hsr) throws IOException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + + RMWSConsts.REPLACE_NODE_TO_LABELS, + HTTPMethods.POST, null, null, newNodeToLabels); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public Response replaceLabelsOnNode(Set newNodeLabelsName, + HttpServletRequest hsr, String nodeId) throws Exception { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + Map paramMap = new HashMap<>(); + paramMap.put(RMWSConsts.LABELS, + newNodeLabelsName.toArray(new String[newNodeLabelsName.size()])); + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES + + "/" + nodeId + "/replace-labels", + HTTPMethods.POST, null, paramMap, null); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public NodeLabelsInfo getClusterNodeLabels(HttpServletRequest hsr) + throws IOException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public NodeLabelsInfo run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.GET_NODE_LABELS, + HTTPMethods.GET, null, null, null); + // YARN RM can answer with Status.OK + if (response.getStatus() == STATUS_OK) { + return response.getEntity(NodeLabelsInfo.class); + } + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public Response addToClusterNodeLabels(NodeLabelsInfo newNodeLabels, + HttpServletRequest hsr) throws Exception { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.ADD_NODE_LABELS, + HTTPMethods.POST, null, null, newNodeLabels); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public Response removeFromCluserNodeLabels(Set oldNodeLabels, + HttpServletRequest hsr) throws Exception { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + Map paramMap = new HashMap<>(); + paramMap.put(RMWSConsts.LABELS, + oldNodeLabels.toArray(new String[oldNodeLabels.size()])); + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + + RMWSConsts.REMOVE_NODE_LABELS, + HTTPMethods.POST, null, paramMap, null); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public NodeLabelsInfo getLabelsOnNode(HttpServletRequest hsr, String nodeId) + throws IOException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public NodeLabelsInfo run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES, + HTTPMethods.GET, nodeId + "/get-labels", null, null); + // YARN RM can answer with Status.OK + if (response.getStatus() == STATUS_OK) { + return response.getEntity(NodeLabelsInfo.class); + } + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public AppPriority getAppPriority(HttpServletRequest hsr, String appId) + throws AuthorizationException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public AppPriority run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, + HTTPMethods.GET, appId + "/" + RMWSConsts.PRIORITY, null, null); + // YARN RM can answer with Status.OK or it throws an exception + if (response.getStatus() == STATUS_OK) { + return response.getEntity(AppPriority.class); + } + RouterWebServiceUtil.retrieveException(response); + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public Response updateApplicationPriority(AppPriority targetPriority, + HttpServletRequest hsr, String appId) throws AuthorizationException, + YarnException, InterruptedException, IOException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, getConf())) { + throw new AuthorizationException( + "The default static user cannot carry out this operation."); + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, + HTTPMethods.PUT, appId + "/" + RMWSConsts.PRIORITY, + hsr.getParameterMap(), targetPriority); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + + } + + @Override + public AppQueue getAppQueue(HttpServletRequest hsr, String appId) + throws AuthorizationException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public AppQueue run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, + HTTPMethods.GET, appId + "/" + RMWSConsts.QUEUE, null, null); + // YARN RM can answer with Status.OK or it throws an exception + if (response.getStatus() == STATUS_OK) { + return response.getEntity(AppQueue.class); + } + RouterWebServiceUtil.retrieveException(response); + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public Response updateAppQueue(AppQueue targetQueue, HttpServletRequest hsr, + String appId) throws AuthorizationException, YarnException, + InterruptedException, IOException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, getConf())) { + throw new AuthorizationException( + "The default static user cannot carry out this operation."); + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, + HTTPMethods.PUT, appId + "/" + RMWSConsts.QUEUE, + hsr.getParameterMap(), targetQueue); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public Response createNewApplication(HttpServletRequest hsr) + throws AuthorizationException, IOException, InterruptedException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + + RMWSConsts.APPS_NEW_APPLICATION, + HTTPMethods.POST, null, null, null); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } + + @Override + public Response submitApplication(ApplicationSubmissionContextInfo newApp, + HttpServletRequest hsr) + throws AuthorizationException, IOException, InterruptedException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, getConf())) { + throw new AuthorizationException( + "The default static user cannot carry out this operation."); + } + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, + HTTPMethods.POST, null, hsr.getParameterMap(), newApp); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public Response postDelegationToken(DelegationToken tokenData, + HttpServletRequest hsr) throws AuthorizationException, IOException, + InterruptedException, Exception { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.DELEGATION_TOKEN, + HTTPMethods.POST, null, null, tokenData); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } + + @Override + public Response postDelegationTokenExpiration(HttpServletRequest hsr) + throws AuthorizationException, IOException, InterruptedException, + Exception { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + + RMWSConsts.DELEGATION_TOKEN_EXPIRATION, + HTTPMethods.POST, null, null, null); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } + + @Override + public Response cancelDelegationToken(HttpServletRequest hsr) + throws AuthorizationException, IOException, InterruptedException, + Exception { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.DELEGATION_TOKEN, + HTTPMethods.DELETE, null, null, null); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } + + @Override + public Response createNewReservation(HttpServletRequest hsr) + throws AuthorizationException, IOException, InterruptedException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_NEW, + HTTPMethods.POST, null, null, null); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + + } + + @Override + public Response submitReservation(ReservationSubmissionRequestInfo resContext, + HttpServletRequest hsr) + throws AuthorizationException, IOException, InterruptedException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, getConf())) { + throw new AuthorizationException( + "The default static user cannot carry out this operation."); + } + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + + RMWSConsts.RESERVATION_SUBMIT, + HTTPMethods.POST, null, hsr.getParameterMap(), resContext); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public Response updateReservation(ReservationUpdateRequestInfo resContext, + HttpServletRequest hsr) + throws AuthorizationException, IOException, InterruptedException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, getConf())) { + throw new AuthorizationException( + "The default static user cannot carry out this operation."); + } + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + + RMWSConsts.RESERVATION_UPDATE, + HTTPMethods.POST, null, hsr.getParameterMap(), resContext); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public Response deleteReservation(ReservationDeleteRequestInfo resContext, + HttpServletRequest hsr) + throws AuthorizationException, IOException, InterruptedException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } + + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, getConf())) { + throw new AuthorizationException( + "The default static user cannot carry out this operation."); + } + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + + RMWSConsts.RESERVATION_DELETE, + HTTPMethods.POST, null, hsr.getParameterMap(), resContext); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + + } + + @Override + public Response listReservation(String queue, String reservationId, + long startTime, long endTime, boolean includeResourceAllocations, + HttpServletRequest hsr) throws Exception { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, getConf())) { + throw new AuthorizationException( + "The default static user cannot carry out this operation."); + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = + RouterWebServiceUtil.invokeRMWebService(webAppAddress, + RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_LIST, + HTTPMethods.GET, null, hsr.getParameterMap(), null); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public AppTimeoutInfo getAppTimeout(HttpServletRequest hsr, String appId, + String type) throws AuthorizationException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, getConf())) { + throw new AuthorizationException( + "The default static user cannot carry out this operation."); + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public AppTimeoutInfo run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, + HTTPMethods.GET, appId + "/" + RMWSConsts.TIMEOUTS + "/" + type, + null, null); + // YARN RM can answer with Status.OK + if (response.getStatus() == STATUS_OK) { + return response.getEntity(AppTimeoutInfo.class); + } + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public AppTimeoutsInfo getAppTimeouts(HttpServletRequest hsr, String appId) + throws AuthorizationException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public AppTimeoutsInfo run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, + HTTPMethods.GET, appId + "/" + RMWSConsts.TIMEOUTS, null, null); + // YARN RM can answer with Status.OK + if (response.getStatus() == STATUS_OK) { + return response.getEntity(AppTimeoutsInfo.class); + } + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public Response updateApplicationTimeout(AppTimeoutInfo appTimeout, + HttpServletRequest hsr, String appId) throws AuthorizationException, + YarnException, InterruptedException, IOException { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } + if (UserGroupInformation.isSecurityEnabled() + && RMWebAppUtil.isStaticUser(callerUGI, getConf())) { + throw new AuthorizationException( + "The default static user cannot carry out this operation."); + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public Response run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, + HTTPMethods.PUT, appId + "/" + RMWSConsts.TIMEOUT, + hsr.getParameterMap(), appTimeout); + return RouterWebServiceUtil.clientResponseToResponse(response); + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public AppAttemptsInfo getAppAttempts(HttpServletRequest hsr, String appId) { + UserGroupInformation callerUGI = + RMWebAppUtil.getCallerUserGroupInformation(hsr, true); + if (callerUGI == null) { + try { + throw new AuthorizationException( + "Unable to obtain user name, user not authenticated"); + } catch (AuthorizationException e) { + return null; + } + } + + try { + return callerUGI.doAs(new PrivilegedExceptionAction() { + @Override + public AppAttemptsInfo run() { + ClientResponse response = RouterWebServiceUtil.invokeRMWebService( + webAppAddress, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, + HTTPMethods.GET, appId + "/" + RMWSConsts.APPATTEMPTS, null, + null); + // YARN RM can answer with Status.OK or it throws an exception + if (response.getStatus() == STATUS_OK) { + return response.getEntity(AppAttemptsInfo.class); + } + RouterWebServiceUtil.retrieveException(response); + return null; + } + }); + } catch (InterruptedException e) { + return null; + } catch (IOException e) { + return null; + } + } + + @Override + public AppAttemptInfo getAppAttempt(HttpServletRequest req, + HttpServletResponse res, String appId, String appAttemptId) { + throw new NotImplementedException(); + } + + @Override + public ContainersInfo getContainers(HttpServletRequest req, + HttpServletResponse res, String appId, String appAttemptId) { + throw new NotImplementedException(); + } + + @Override + public ContainerInfo getContainer(HttpServletRequest req, + HttpServletResponse res, String appId, String appAttemptId, + String containerId) { + throw new NotImplementedException(); + } + + @Override + public void setNextInterceptor(RESTRequestInterceptor next) { + throw new YarnRuntimeException("setNextInterceptor is being called on " + + "DefaultRequestInterceptorREST, which should be the last one " + + "in the chain. Check if the interceptor pipeline configuration " + + "is correct"); + } + +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/HTTPMethods.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/HTTPMethods.java new file mode 100644 index 0000000..ed24297 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/HTTPMethods.java @@ -0,0 +1,24 @@ +/** + * 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; + +public enum HTTPMethods { + + GET, PUT, DELETE, POST +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RESTRequestInterceptor.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RESTRequestInterceptor.java new file mode 100644 index 0000000..48dd199 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RESTRequestInterceptor.java @@ -0,0 +1,90 @@ +/** + * 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 javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServiceProtocol; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo; +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; + +/** + * Defines the contract to be implemented by the request intercepter classes, + * that can be used to intercept and inspect messages sent from the client to + * the resource manager server. + */ +public interface RESTRequestInterceptor + extends RMWebServiceProtocol, Configurable { + + /** + * This method is called for initializing the intercepter. This is guaranteed + * to be called only once in the lifetime of this instance. + * + * @param user + * + * @param ctx + */ + public abstract void init(String user); + + /** + * This method is called to release the resources held by the intercepter. + * This will be called when the application pipeline is being destroyed. The + * concrete implementations should dispose the resources and forward the + * request to the next intercepter, if any. + */ + public abstract void shutdown(); + + /** + * Sets the next intercepter in the pipeline. The concrete implementation of + * this interface should always pass the request to the nextInterceptor after + * inspecting the message. The last intercepter in the chain is responsible to + * send the messages to the resource manager service and so the last + * intercepter will not receive this method call. + * + * @param nextInterceptor + */ + public abstract void setNextInterceptor( + RESTRequestInterceptor nextInterceptor); + + /** + * Returns the next intercepter in the chain. + * + * @return the next intercepter in the chain + */ + public abstract RESTRequestInterceptor getNextInterceptor(); + + // Functions implemented in WebService + + public abstract AppAttemptsInfo getAppAttempts(HttpServletRequest hsr, + String appId); + + public abstract AppAttemptInfo getAppAttempt(HttpServletRequest req, + HttpServletResponse res, String appId, String appAttemptId); + + public abstract ContainersInfo getContainers(HttpServletRequest req, + HttpServletResponse res, String appId, String appAttemptId); + + public abstract ContainerInfo getContainer(HttpServletRequest req, + HttpServletResponse res, String appId, String appAttemptId, + String containerId); +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterRedirectionFilter.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterRedirectionFilter.java new file mode 100644 index 0000000..9cfcdc5 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterRedirectionFilter.java @@ -0,0 +1,62 @@ +/** + * 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 javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import javax.servlet.http.HttpServletResponse; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.webapp.util.WebAppUtils; + +@Private +@Unstable +public class RouterRedirectionFilter implements Filter { + + private String webAppAddress; + + @Override + public void init(FilterConfig filterConfig) throws ServletException { + webAppAddress = + WebAppUtils.getRouterWebAppURLWithScheme(new Configuration()); + } + + /** + * {@inheritDoc} + */ + @Override + public void doFilter(ServletRequest request, ServletResponse response, + FilterChain filterChain) throws IOException, ServletException { + ((HttpServletResponse) response).sendRedirect(webAppAddress); + } + + @Override + public void destroy() { + webAppAddress = null; + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebApp.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebApp.java new file mode 100644 index 0000000..5436bad --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebApp.java @@ -0,0 +1,48 @@ +/** + * 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 org.apache.hadoop.yarn.server.resourcemanager.webapp.JAXBContextResolver; +import org.apache.hadoop.yarn.server.router.Router; +import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; +import org.apache.hadoop.yarn.webapp.WebApp; +import org.apache.hadoop.yarn.webapp.YarnWebParams; + +/** + * The Router webapp. + */ +public class RouterWebApp extends WebApp implements YarnWebParams { + private Router router; + + public RouterWebApp(Router router) { + this.router = router; + } + + @Override + public void setup() { + bind(JAXBContextResolver.class); + bind(RouterWebServices.class); + bind(GenericExceptionHandler.class); + bind(RouterWebApp.class).toInstance(this); + + if (router != null) { + bind(Router.class).toInstance(router); + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServiceUtil.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServiceUtil.java new file mode 100644 index 0000000..16fc9a7 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServiceUtil.java @@ -0,0 +1,138 @@ +/** + * 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.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.MultivaluedMap; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.ResponseBuilder; + +import org.apache.hadoop.yarn.webapp.BadRequestException; +import org.apache.hadoop.yarn.webapp.ForbiddenException; +import org.apache.hadoop.yarn.webapp.NotFoundException; + +import com.sun.jersey.api.ConflictException; +import com.sun.jersey.api.client.Client; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.api.client.WebResource.Builder; +import com.sun.jersey.core.util.MultivaluedMapImpl; + +/** + * The Router webservice util class. + */ +public class RouterWebServiceUtil { + + /** + * Performs an invocation of the the remote RMWebService. + */ + public static ClientResponse invokeRMWebService(String webApp, String path, + HTTPMethods method, String additionalPath, + Map queryParams, Object formParam) { + Client client = Client.create(); + + WebResource webResource = client.resource(webApp).path(path); + + if (additionalPath != null && !additionalPath.isEmpty()) { + webResource = webResource.path(additionalPath); + } + + if (queryParams != null && !queryParams.isEmpty()) { + MultivaluedMap paramMap = new MultivaluedMapImpl(); + + for (Entry param : queryParams.entrySet()) { + String[] values = param.getValue(); + for (int i = 0; i < values.length; i++) { + paramMap.add(param.getKey(), values[i]); + } + } + webResource = webResource.queryParams(paramMap); + } + + Builder builder = null; + if (formParam != null) { + builder = webResource.entity(formParam, MediaType.APPLICATION_XML); + builder = builder.accept(MediaType.APPLICATION_XML); + } else { + builder = webResource.accept(MediaType.APPLICATION_XML); + } + + ClientResponse response = null; + + switch (method) { + case DELETE: + response = builder.delete(ClientResponse.class); + break; + case GET: + response = builder.get(ClientResponse.class); + break; + case POST: + response = builder.post(ClientResponse.class); + break; + case PUT: + response = builder.put(ClientResponse.class); + break; + default: + break; + } + + return response; + } + + public static Response clientResponseToResponse(ClientResponse r) { + if (r == null) { + return null; + } + // copy the status code + ResponseBuilder rb = Response.status(r.getStatus()); + // copy all the headers + for (Entry> entry : r.getHeaders().entrySet()) { + for (String value : entry.getValue()) { + rb.header(entry.getKey(), value); + } + } + // copy the entity + rb.entity(r.getEntityInputStream()); + // return the response + return rb.build(); + } + + public static void retrieveException(ClientResponse response) { + String serverErrorMsg = response.getEntity(String.class); + int status = response.getStatus(); + if (status == 400) { + throw new BadRequestException(serverErrorMsg); + } + if (status == 403) { + throw new ForbiddenException(serverErrorMsg); + } + if (status == 404) { + throw new NotFoundException(serverErrorMsg); + } + if (status == 409) { + throw new ConflictException(serverErrorMsg); + } + + } + +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java 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..2b0cbd9 --- /dev/null +++ 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,864 @@ +/** + * 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; + +@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 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( + 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 getInterceptorClassNames(Configuration conf) { + String configuredInterceptorClassNames = + conf.get(YarnConfiguration.ROUTER_WEBAPP_INTERCEPTOR_CLASS_PIPELINE, + YarnConfiguration.DEFAULT_ROUTER_WEBAPP_INTERCEPTOR_CLASS); + + List interceptorClassNames = new ArrayList(); + Collection 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 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 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 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 applicationTypes, + @QueryParam(RMWSConsts.APPLICATION_TAGS) Set applicationTags) { + init(); + RequestInterceptorChainWrapper pipeline = getInterceptorChain(); + return pipeline.getRootInterceptor().getApps(hsr, stateQuery, statesQuery, + finalStatusQuery, userQuery, queueQuery, count, startedBegin, + startedEnd, finishBegin, finishEnd, applicationTypes, applicationTags); + } + + @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 stateQueries, + @QueryParam(RMWSConsts.APPLICATION_TYPES) Set 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) { + init(); + RequestInterceptorChainWrapper pipeline = getInterceptorChain(); + return pipeline.getRootInterceptor().getApp(hsr, appId); + } + + @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 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 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 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; + } + +} diff --git 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 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 +++ 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; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.java 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..328578e --- /dev/null +++ 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; + + protected 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() { + @Override + public ClusterInfo run() throws Exception { + return routerWebService.get(); + } + }); + } + + protected ClusterInfo getClusterInfo(String user) + throws IOException, InterruptedException { + return UserGroupInformation.createRemoteUser(user) + .doAs(new PrivilegedExceptionAction() { + @Override + public ClusterInfo run() throws Exception { + return routerWebService.getClusterInfo(); + } + }); + } + + protected ClusterMetricsInfo getClusterMetricsInfo(String user) + throws IOException, InterruptedException { + return UserGroupInformation.createRemoteUser(user) + .doAs(new PrivilegedExceptionAction() { + @Override + public ClusterMetricsInfo run() throws Exception { + return routerWebService.getClusterMetricsInfo(); + } + }); + } + + protected SchedulerTypeInfo getSchedulerInfo(String user) + throws IOException, InterruptedException { + return UserGroupInformation.createRemoteUser(user) + .doAs(new PrivilegedExceptionAction() { + @Override + public SchedulerTypeInfo run() throws Exception { + return routerWebService.getSchedulerInfo(); + } + }); + } + + protected String dumpSchedulerLogs(String user) + throws IOException, InterruptedException { + return UserGroupInformation.createRemoteUser(user) + .doAs(new PrivilegedExceptionAction() { + @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() { + @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() { + @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() { + @Override + public AppsInfo run() throws Exception { + return routerWebService.getApps(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() { + @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() { + @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() { + @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() { + @Override + public AppInfo run() throws Exception { + return routerWebService.getApp(null, null); + } + }); + } + + protected AppState getAppState(String user) + throws IOException, InterruptedException { + return UserGroupInformation.createRemoteUser(user) + .doAs(new PrivilegedExceptionAction() { + @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() { + @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() { + @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() { + @Override + public LabelsToNodesInfo run() throws Exception { + return routerWebService.getLabelsToNodes(null); + } + }); + } + + protected Response replaceLabelsOnNodes(String user) throws Exception { + return UserGroupInformation.createRemoteUser(user) + .doAs(new PrivilegedExceptionAction() { + @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() { + @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() { + @Override + public NodeLabelsInfo run() throws Exception { + return routerWebService.getClusterNodeLabels(null); + } + }); + } + + protected Response addToClusterNodeLabels(String user) throws Exception { + return UserGroupInformation.createRemoteUser(user) + .doAs(new PrivilegedExceptionAction() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @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() { + @Override + public RequestInterceptorChainWrapper run() throws Exception { + return routerWebService.getInterceptorChain(); + } + }); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/JavaProcess.java 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..1f3ab3b --- /dev/null +++ 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,51 @@ +/** + * 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; + +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); + + process = builder.start(); + + Thread.sleep(1000); + } + + public void stop() throws InterruptedException { + if (process != null) { + process.destroy(); + process.waitFor(); + process.exitValue(); + } + } + +} \ No newline at end of file diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockRESTRequestInterceptor.java 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..816d9e0 --- /dev/null +++ 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,336 @@ +/** + * 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; + +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 statesQuery, String finalStatusQuery, String userQuery, + String queueQuery, String count, String startedBegin, String startedEnd, + String finishBegin, String finishEnd, Set applicationTypes, + Set applicationTags) { + 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 stateQueries, Set typeQueries) { + return new ApplicationStatisticsInfo(); + } + + @Override + public AppInfo getApp(HttpServletRequest hsr, String appId) { + 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 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 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 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 diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/PassThroughRESTRequestInterceptor.java 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..6a72dbd --- /dev/null +++ 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,333 @@ +/** + * 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; + +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 statesQuery, String finalStatusQuery, String userQuery, + String queueQuery, String count, String startedBegin, String startedEnd, + String finishBegin, String finishEnd, Set applicationTypes, + Set applicationTags) { + return getNextInterceptor().getApps(hsr, stateQuery, statesQuery, + finalStatusQuery, userQuery, queueQuery, count, startedBegin, + startedEnd, finishBegin, finishEnd, applicationTypes, applicationTags); + } + + @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 stateQueries, Set typeQueries) { + return getNextInterceptor().getAppStatistics(hsr, stateQueries, + typeQueries); + } + + @Override + public AppInfo getApp(HttpServletRequest hsr, String appId) { + return getNextInterceptor().getApp(hsr, appId); + } + + @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 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 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 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); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServices.java 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..0a2f39a --- /dev/null +++ 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,359 @@ +/** + * 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test class to validate the WebService interceptor model inside the Router. + */ +public class TestRouterWebServices extends BaseRouterWebServicesTest { + + private static final Logger LOG = + LoggerFactory.getLogger(TestRouterWebServices.class); + + /** + * Test that all requests get forwarded to the last interceptor in the chain + * get back the responses. + */ + @Test + public void testRouterWebServicesE2E() throws Exception { + String user = "test1"; + + LOG.info("testRouterWebServicesE2E - Get"); + + ClusterInfo clusterInfo = get(user); + Assert.assertNotNull(clusterInfo); + + LOG.info("testRouterWebServicesE2E - Get Cluster Info"); + + ClusterInfo clusterInfo2 = getClusterInfo(user); + Assert.assertNotNull(clusterInfo2); + + LOG.info("testRouterWebServicesE2E - Get Cluster Metrics Info"); + + ClusterMetricsInfo clusterMetricsInfo = getClusterMetricsInfo(user); + Assert.assertNotNull(clusterMetricsInfo); + + LOG.info("testRouterWebServicesE2E - Get SchedulerInfo"); + + SchedulerTypeInfo schedulerTypeInfo = getSchedulerInfo(user); + Assert.assertNotNull(schedulerTypeInfo); + + LOG.info("testRouterWebServicesE2E - Dump Scheduler Logs"); + + String dumpResult = dumpSchedulerLogs(user); + Assert.assertNotNull(dumpResult); + + LOG.info("testRouterWebServicesE2E - Get Nodes"); + + NodesInfo nodesInfo = getNodes(user); + Assert.assertNotNull(nodesInfo); + + LOG.info("testRouterWebServicesE2E - Get Node"); + + NodeInfo nodeInfo = getNode(user); + Assert.assertNotNull(nodeInfo); + + LOG.info("testRouterWebServicesE2E - Get Apps"); + + AppsInfo appsInfo = getApps(user); + Assert.assertNotNull(appsInfo); + + LOG.info("testRouterWebServicesE2E - Get Activities"); + + ActivitiesInfo activitiesInfo = getActivities(user); + Assert.assertNotNull(activitiesInfo); + + LOG.info("testRouterWebServicesE2E - Get AppActivities"); + + AppActivitiesInfo appActiviesInfo = getAppActivities(user); + Assert.assertNotNull(appActiviesInfo); + + LOG.info("testRouterWebServicesE2E - Get AppStatistics"); + + ApplicationStatisticsInfo applicationStatisticsInfo = + getAppStatistics(user); + Assert.assertNotNull(applicationStatisticsInfo); + + LOG.info("testRouterWebServicesE2E - Get App"); + + AppInfo appInfo = getApp(user); + Assert.assertNotNull(appInfo); + + LOG.info("testRouterWebServicesE2E - Get AppState"); + + AppState appState = getAppState(user); + Assert.assertNotNull(appState); + + LOG.info("testRouterWebServicesE2E - Update AppState"); + + Response response = updateAppState(user); + Assert.assertNotNull(response); + + LOG.info("testRouterWebServicesE2E - Get Node To Labels"); + + NodeToLabelsInfo nodeToLabelsInfo = getNodeToLabels(user); + Assert.assertNotNull(nodeToLabelsInfo); + + LOG.info("testRouterWebServicesE2E - Get Labels To Nodes"); + + LabelsToNodesInfo labelsToNodesInfo = getLabelsToNodes(user); + Assert.assertNotNull(labelsToNodesInfo); + + LOG.info("testRouterWebServicesE2E - Replace Labels On Nodes"); + + Response response2 = replaceLabelsOnNodes(user); + Assert.assertNotNull(response2); + + LOG.info("testRouterWebServicesE2E - Replace Labels On Node"); + + Response response3 = replaceLabelsOnNode(user); + Assert.assertNotNull(response3); + + LOG.info("testRouterWebServicesE2E - Get Cluster NodeLabels"); + + NodeLabelsInfo nodeLabelsInfo = getClusterNodeLabels(user); + Assert.assertNotNull(nodeLabelsInfo); + + LOG.info("testRouterWebServicesE2E - Add To Cluster NodeLabels"); + + Response response4 = addToClusterNodeLabels(user); + Assert.assertNotNull(response4); + + LOG.info("testRouterWebServicesE2E - Remove From Cluser NodeLabels"); + + Response response5 = removeFromCluserNodeLabels(user); + Assert.assertNotNull(response5); + + LOG.info("testRouterWebServicesE2E - Get Labels On Node"); + + NodeLabelsInfo nodeLabelsInfo2 = getLabelsOnNode(user); + Assert.assertNotNull(nodeLabelsInfo2); + + LOG.info("testRouterWebServicesE2E - Get AppPriority"); + + AppPriority appPriority = getAppPriority(user); + Assert.assertNotNull(appPriority); + + LOG.info("testRouterWebServicesE2E - Update Application Priority"); + + Response response6 = updateApplicationPriority(user); + Assert.assertNotNull(response6); + + LOG.info("testRouterWebServicesE2E - Get AppQueue"); + + AppQueue appQueue = getAppQueue(user); + Assert.assertNotNull(appQueue); + + LOG.info("testRouterWebServicesE2E - Update AppQueue"); + + Response response7 = updateAppQueue(user); + Assert.assertNotNull(response7); + + LOG.info("testRouterWebServicesE2E - Create New Application"); + + Response response8 = createNewApplication(user); + Assert.assertNotNull(response8); + + LOG.info("testRouterWebServicesE2E - Submit Application"); + + Response response9 = submitApplication(user); + Assert.assertNotNull(response9); + + LOG.info("testRouterWebServicesE2E - Post Delegation Token"); + + Response response10 = postDelegationToken(user); + Assert.assertNotNull(response10); + + LOG.info("testRouterWebServicesE2E - Post Delegation Token Expiration"); + + Response response11 = postDelegationTokenExpiration(user); + Assert.assertNotNull(response11); + + LOG.info("testRouterWebServicesE2E - Cancel Delegation Token"); + + Response response12 = cancelDelegationToken(user); + Assert.assertNotNull(response12); + + LOG.info("testRouterWebServicesE2E - Create New Reservation"); + + Response response13 = createNewReservation(user); + Assert.assertNotNull(response13); + + LOG.info("testRouterWebServicesE2E - Submit Reservation"); + + Response response14 = submitReservation(user); + Assert.assertNotNull(response14); + + LOG.info("testRouterWebServicesE2E - Update Reservation"); + + Response response15 = updateReservation(user); + Assert.assertNotNull(response15); + + LOG.info("testRouterWebServicesE2E - Delete Reservation"); + + Response response16 = deleteReservation(user); + Assert.assertNotNull(response16); + + LOG.info("testRouterWebServicesE2E - List Reservation"); + + Response response17 = listReservation(user); + Assert.assertNotNull(response17); + + LOG.info("testRouterWebServicesE2E - Get AppTimeout"); + + AppTimeoutInfo appTimeoutInfo = getAppTimeout(user); + Assert.assertNotNull(appTimeoutInfo); + + LOG.info("testRouterWebServicesE2E - Get AppTimeouts"); + + AppTimeoutsInfo appTimeoutsInfo = getAppTimeouts(user); + Assert.assertNotNull(appTimeoutsInfo); + + LOG.info("testRouterWebServicesE2E - Update Application Timeout"); + + Response response18 = updateApplicationTimeout(user); + Assert.assertNotNull(response18); + + LOG.info("testRouterWebServicesE2E - Get AppAttempts"); + + AppAttemptsInfo appAttemptsInfo = getAppAttempts(user); + Assert.assertNotNull(appAttemptsInfo); + + LOG.info("testRouterWebServicesE2E - Get AppAttempt"); + + AppAttemptInfo appAttemptInfo = getAppAttempt(user); + Assert.assertNotNull(appAttemptInfo); + + LOG.info("testRouterWebServicesE2E - Get Containers"); + + ContainersInfo containersInfo = getContainers(user); + Assert.assertNotNull(containersInfo); + + LOG.info("testRouterWebServicesE2E - Get Container"); + + 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 pipelines = + routerWebService.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); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java 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..9482bb4 --- /dev/null +++ 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,1427 @@ +/** + * 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.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.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.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.dao.AppsInfo; +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.ClientResponse; +import com.sun.jersey.api.client.WebResource; + +public class TestRouterWebServicesREST { + + private static Router router; + public final static int TEST_MAX_CACHE_SIZE = 10; + private String userName = "test"; + + private static JavaProcess rm; + private static JavaProcess nm; + + 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; + + @BeforeClass + public static void setUp() throws Exception { + conf = new YarnConfiguration(); + conf.setBoolean("mockrm.webapp.enabled", true); + router = new Router(); + router.init(conf); + router.start(); + rm = new JavaProcess(ResourceManager.class); + + // Add 1 node + nm = new JavaProcess(NodeManager.class); + } + + @AfterClass + public static void stop() throws Exception { + router.stop(); + rm.stop(); + nm.stop(); + } + + @Test + public void testInfoXML() throws JSONException, Exception { + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + ClusterInfo ci = response.getEntity(ClusterInfo.class); + ClusterInfo ci2 = response2.getEntity(ClusterInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testClusterInfoXML() throws JSONException, Exception { + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.INFO); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.INFO); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + ClusterInfo ci = response.getEntity(ClusterInfo.class); + ClusterInfo ci2 = response2.getEntity(ClusterInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testMetricsInfoXML() throws JSONException, Exception { + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.METRICS); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.METRICS); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + ClusterMetricsInfo ci = + response.getEntity(ClusterMetricsInfo.class); + ClusterMetricsInfo ci2 = + response2.getEntity(ClusterMetricsInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testSchedulerInfoXML() throws JSONException, Exception { + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + SchedulerTypeInfo ci = + response.getEntity(SchedulerTypeInfo.class); + SchedulerTypeInfo ci2 = + response2.getEntity(SchedulerTypeInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testNodesXML() throws JSONException, Exception { + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES) + .queryParam(RMWSConsts.STATES, "RUNNING"); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES) + .queryParam(RMWSConsts.STATES, "RUNNING"); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + NodesInfo ci = response.getEntity(NodesInfo.class); + NodesInfo ci2 = response2.getEntity(NodesInfo.class); + Assert.assertEquals(ci.getNodes().size(), ci2.getNodes().size()); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testNodeXML() throws JSONException, Exception { + + String nodeId = getNodeId(); + + Client clientToRouter = Client.create(); + WebResource toRouter = clientToRouter + .resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES).path(nodeId); + + Client clientToRM = Client.create(); + WebResource toRM = clientToRM + .resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES).path(nodeId); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + NodeInfo ci = response.getEntity(NodeInfo.class); + NodeInfo ci2 = response2.getEntity(NodeInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testActiviesXML() throws JSONException, Exception { + + Client clientToRouter = Client.create(); + WebResource toRouter = clientToRouter + .resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_ACTIVITIES); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)).path( + RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_ACTIVITIES); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + ActivitiesInfo ci = response.getEntity(ActivitiesInfo.class); + ActivitiesInfo ci2 = response2.getEntity(ActivitiesInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testAppActivitiesXML() throws JSONException, Exception { + + String appId = submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + + RMWSConsts.SCHEDULER_APP_ACTIVITIES) + .queryParam(RMWSConsts.APP_ID, appId); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + + RMWSConsts.SCHEDULER_APP_ACTIVITIES) + .queryParam(RMWSConsts.APP_ID, appId); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + AppActivitiesInfo ci = + response.getEntity(AppActivitiesInfo.class); + AppActivitiesInfo ci2 = + response2.getEntity(AppActivitiesInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testAppStatisticsXML() throws JSONException, Exception { + + submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APP_STATISTICS) + .queryParam(RMWSConsts.STATES, "RUNNING"); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APP_STATISTICS) + .queryParam(RMWSConsts.STATES, "RUNNING"); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + ApplicationStatisticsInfo ci = + response.getEntity(ApplicationStatisticsInfo.class); + ApplicationStatisticsInfo ci2 = + response2.getEntity(ApplicationStatisticsInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testDumpSchedulerLogsXML() throws JSONException, Exception { + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_LOGS) + .queryParam(RMWSConsts.TIME, "1"); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .post(ClientResponse.class); + + if (response.getStatus() == STATUS_BADREQUEST) { + String ci = response.getEntity(String.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testNewApplicationXML() throws JSONException, Exception { + + Client clientToRouter = Client.create(); + WebResource toRouter = clientToRouter + .resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS_NEW_APPLICATION); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .post(ClientResponse.class); + if (response.getStatus() == STATUS_OK) { + NewApplication ci = response.getEntity(NewApplication.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testSubmitApplicationXML() throws JSONException, Exception { + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ApplicationSubmissionContextInfo context = + new ApplicationSubmissionContextInfo(); + context.setApplicationId(getNewApplicationId().getApplicationId()); + + ClientResponse response = toRouter + .entity(context, MediaType.APPLICATION_XML) + .accept(MediaType.APPLICATION_XML).post(ClientResponse.class); + if (response.getStatus() == STATUS_ACCEPTED) { + String ci = response.getEntity(String.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testAppsXML() throws JSONException, Exception { + + submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + AppsInfo ci = response.getEntity(AppsInfo.class); + AppsInfo ci2 = response2.getEntity(AppsInfo.class); + Assert.assertEquals(ci.getApps().size(), ci2.getApps().size()); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testAppXML() throws JSONException, Exception { + + String appId = submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + AppInfo ci = response.getEntity(AppInfo.class); + AppInfo ci2 = response2.getEntity(AppInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testAppAttemptXML() throws JSONException, Exception { + + String appId = submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.ATTEMPTS); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.ATTEMPTS); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + AppAttemptsInfo ci = response.getEntity(AppAttemptsInfo.class); + AppAttemptsInfo ci2 = response2.getEntity(AppAttemptsInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testAppStateXML() throws JSONException, Exception { + + String appId = submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.STATE); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.STATE); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + AppState ci = response.getEntity(AppState.class); + AppState ci2 = response2.getEntity(AppState.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testUpdateAppStateXML() throws JSONException, Exception { + + String appId = submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.STATE); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + AppState appState = new AppState("KILLED"); + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .entity(appState, MediaType.APPLICATION_XML) + .put(ClientResponse.class); + if (response.getStatus() == STATUS_ACCEPTED) { + AppState ci = response.getEntity(AppState.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testAppPriorityXML() throws JSONException, Exception { + + String appId = submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.PRIORITY); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.PRIORITY); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + AppPriority ci = response.getEntity(AppPriority.class); + AppPriority ci2 = response2.getEntity(AppPriority.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testUpdateAppPriorityXML() throws JSONException, Exception { + + String appId = submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.PRIORITY); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + AppPriority appPriority = new AppPriority(1); + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .entity(appPriority, MediaType.APPLICATION_XML) + .put(ClientResponse.class); + if (response.getStatus() == STATUS_OK) { + AppPriority ci = response.getEntity(AppPriority.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testAppQueueXML() throws JSONException, Exception { + + String appId = submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.QUEUE); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.QUEUE); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + AppQueue ci = response.getEntity(AppQueue.class); + AppQueue ci2 = response2.getEntity(AppQueue.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testUpdateAppQueueXML() throws JSONException, Exception { + + String appId = submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.QUEUE); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + AppQueue appQueue = new AppQueue("default"); + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .entity(appQueue, MediaType.APPLICATION_XML) + .put(ClientResponse.class); + if (response.getStatus() == STATUS_OK) { + AppQueue ci = response.getEntity(AppQueue.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testAppTimeoutsXML() throws JSONException, Exception { + + String appId = submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.TIMEOUTS); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.TIMEOUTS); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + AppTimeoutsInfo ci = response.getEntity(AppTimeoutsInfo.class); + AppTimeoutsInfo ci2 = response2.getEntity(AppTimeoutsInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testAppTimeoutXML() throws JSONException, Exception { + + String appId = submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.TIMEOUTS).path("LIFETIME"); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.TIMEOUTS).path("LIFETIME"); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + AppTimeoutInfo ci = response.getEntity(AppTimeoutInfo.class); + AppTimeoutInfo ci2 = response2.getEntity(AppTimeoutInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testUpdateAppTimeoutsXML() throws JSONException, Exception { + + String appId = submitApplication(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS).path(appId) + .path(RMWSConsts.TIMEOUT); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + // Create a bad request + AppTimeoutInfo appTimeoutInfo = new AppTimeoutInfo(); + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .entity(appTimeoutInfo, MediaType.APPLICATION_XML) + .put(ClientResponse.class); + if (response.getStatus() == STATUS_BADREQUEST) { + String ci = response.getEntity(String.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + return null; + } + }); + } + + @Test + public void testNewReservationXML() throws JSONException, Exception { + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_NEW); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .post(ClientResponse.class); + if (response.getStatus() == STATUS_OK) { + NewReservation ci = response.getEntity(NewReservation.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testSubmitReservationXML() throws JSONException, Exception { + + Client clientToRouter = Client.create(); + WebResource toRouter = clientToRouter + .resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_SUBMIT); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ReservationSubmissionRequestInfo context = + new ReservationSubmissionRequestInfo(); + context.setReservationId(getNewReservationId().getReservationId()); + // ReservationDefinition is null + ClientResponse response = toRouter + .entity(context, MediaType.APPLICATION_XML) + .accept(MediaType.APPLICATION_XML).post(ClientResponse.class); + if (response.getStatus() == STATUS_BADREQUEST) { + String ci = response.getEntity(String.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testUpdateReservationXML() throws JSONException, Exception { + + String reservationId = getNewReservationId().getReservationId(); + + Client clientToRouter = Client.create(); + WebResource toRouter = clientToRouter + .resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_UPDATE); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ReservationUpdateRequestInfo context = + new ReservationUpdateRequestInfo(); + context.setReservationId(reservationId); + + ClientResponse response = toRouter + .entity(context, MediaType.APPLICATION_XML) + .accept(MediaType.APPLICATION_XML).post(ClientResponse.class); + if (response.getStatus() == STATUS_BADREQUEST) { + String ci = response.getEntity(String.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testDeleteReservationXML() throws JSONException, Exception { + + String reservationId = getNewReservationId().getReservationId(); + + Client clientToRouter = Client.create(); + WebResource toRouter = clientToRouter + .resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_DELETE); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ReservationDeleteRequestInfo context = + new ReservationDeleteRequestInfo(); + context.setReservationId(reservationId); + + ClientResponse response = toRouter + .entity(context, MediaType.APPLICATION_XML) + .accept(MediaType.APPLICATION_XML).post(ClientResponse.class); + if (response.getStatus() == STATUS_BADREQUEST) { + String ci = response.getEntity(String.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testGetNodeToLabelsXML() throws JSONException, Exception { + + Client clientToRouter = Client.create(); + WebResource toRouter = clientToRouter + .resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.GET_NODE_TO_LABELS); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)).path( + RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.GET_NODE_TO_LABELS); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + NodeToLabelsInfo ci = response.getEntity(NodeToLabelsInfo.class); + NodeToLabelsInfo ci2 = + response2.getEntity(NodeToLabelsInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testGetClusterNodeLabelsXML() throws JSONException, Exception { + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.GET_NODE_LABELS); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.GET_NODE_LABELS); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + NodeLabelsInfo ci = response.getEntity(NodeLabelsInfo.class); + NodeLabelsInfo ci2 = response2.getEntity(NodeLabelsInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testGetLabelsOnNodeXML() throws JSONException, Exception { + + String nodeId = getNodeId(); + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES) + .path(nodeId).path(RMWSConsts.GET_LABELS); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES) + .path(nodeId).path(RMWSConsts.GET_LABELS); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + NodeLabelsInfo ci = response.getEntity(NodeLabelsInfo.class); + NodeLabelsInfo ci2 = response2.getEntity(NodeLabelsInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + // FIXME + @Test + public void testGetLabelsMappingXML() throws JSONException, Exception { + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.LABEL_MAPPINGS); + + Client clientToRM = Client.create(); + WebResource toRM = + clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.LABEL_MAPPINGS); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + ClientResponse response2 = toRM.accept(MediaType.APPLICATION_XML) + .get(ClientResponse.class); + if (response.getStatus() == STATUS_OK + && response2.getStatus() == STATUS_OK) { + LabelsToNodesInfo ci = + response.getEntity(LabelsToNodesInfo.class); + LabelsToNodesInfo ci2 = + response2.getEntity(LabelsToNodesInfo.class); + Assert.assertNotNull(ci); + Assert.assertNotNull(ci2); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testAddToClusterNodeLabelsXML() throws JSONException, Exception { + + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.ADD_NODE_LABELS); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + List nodeLabels = new ArrayList(); + nodeLabels.add(NodeLabel.newInstance("default")); + NodeLabelsInfo context = new NodeLabelsInfo(nodeLabels); + ClientResponse response = toRouter + .entity(context, MediaType.APPLICATION_XML) + .accept(MediaType.APPLICATION_XML).post(ClientResponse.class); + if (response.getStatus() == STATUS_OK) { + String ci = response.getEntity(String.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testRemoveFromCluserNodeLabelsXML() + throws JSONException, Exception { + + addNodeLabel(); + Client clientToRouter = Client.create(); + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path( + RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.REMOVE_NODE_LABELS) + .queryParam(RMWSConsts.LABELS, "default"); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .post(ClientResponse.class); + if (response.getStatus() == STATUS_OK) { + String ci = response.getEntity(String.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + + } + + @Test + public void testReplaceLabelsOnNodesXML() throws JSONException, Exception { + addNodeLabel(); + Client clientToRouter = Client.create(); + + WebResource toRouter = clientToRouter + .resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)).path( + RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.REPLACE_NODE_TO_LABELS); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + NodeToLabelsEntryList context = new NodeToLabelsEntryList(); + ClientResponse response = toRouter + .entity(context, MediaType.APPLICATION_XML) + .accept(MediaType.APPLICATION_XML).post(ClientResponse.class); + if (response.getStatus() == STATUS_OK) { + String ci = response.getEntity(String.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + @Test + public void testReplaceLabelsOnNodeXML() throws JSONException, Exception { + addNodeLabel(); + Client clientToRouter = Client.create(); + + WebResource toRouter = + clientToRouter.resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)) + .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES + "/" + + getNodeId() + "/replace-labels") + .queryParam(RMWSConsts.LABELS, "default"); + + UserGroupInformation.createRemoteUser(userName) + .doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + ClientResponse response = toRouter.accept(MediaType.APPLICATION_XML) + .post(ClientResponse.class); + if (response.getStatus() == STATUS_OK) { + String ci = response.getEntity(String.class); + Assert.assertNotNull(ci); + } else { + Assert.fail(); + } + + return null; + } + }); + } + + 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 nodeLabels = new ArrayList(); + 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); + } + +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/log4j.properties 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 +++ 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 diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml new file mode 100644 index 0000000..fc5fc96 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml @@ -0,0 +1,26 @@ + + + + + + + yarn.resourcemanager.reservation-system.enable + true + + + yarn.node-labels.enabled + true + + \ No newline at end of file