|
@@ -25,10 +25,17 @@ import static org.junit.Assume.assumeTrue;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.io.StringReader;
|
|
import java.io.StringReader;
|
|
import java.io.StringWriter;
|
|
import java.io.StringWriter;
|
|
|
|
+import java.net.URI;
|
|
|
|
+import java.util.ArrayList;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
import java.util.Collection;
|
|
import java.util.Collection;
|
|
import java.util.Enumeration;
|
|
import java.util.Enumeration;
|
|
|
|
+import java.util.HashMap;
|
|
|
|
+import java.util.HashSet;
|
|
|
|
+import java.util.List;
|
|
|
|
+import java.util.Map;
|
|
import java.util.Properties;
|
|
import java.util.Properties;
|
|
|
|
+import java.util.Set;
|
|
|
|
|
|
import javax.servlet.FilterConfig;
|
|
import javax.servlet.FilterConfig;
|
|
import javax.servlet.ServletException;
|
|
import javax.servlet.ServletException;
|
|
@@ -38,9 +45,15 @@ import javax.xml.parsers.DocumentBuilder;
|
|
import javax.xml.parsers.DocumentBuilderFactory;
|
|
import javax.xml.parsers.DocumentBuilderFactory;
|
|
import javax.xml.parsers.ParserConfigurationException;
|
|
import javax.xml.parsers.ParserConfigurationException;
|
|
|
|
|
|
|
|
+import org.apache.commons.codec.binary.Base64;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
|
|
import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
|
|
import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
|
|
import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.LocalResource;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
|
|
import org.apache.hadoop.yarn.api.records.QueueACL;
|
|
import org.apache.hadoop.yarn.api.records.QueueACL;
|
|
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
@@ -54,7 +67,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CredentialsInfo;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LocalResourceInfo;
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
|
|
+import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
|
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
|
|
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
|
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
|
|
import org.codehaus.jettison.json.JSONException;
|
|
import org.codehaus.jettison.json.JSONException;
|
|
@@ -80,6 +97,7 @@ import com.sun.jersey.api.client.Client;
|
|
import com.sun.jersey.api.client.ClientResponse;
|
|
import com.sun.jersey.api.client.ClientResponse;
|
|
import com.sun.jersey.api.client.ClientResponse.Status;
|
|
import com.sun.jersey.api.client.ClientResponse.Status;
|
|
import com.sun.jersey.api.client.WebResource;
|
|
import com.sun.jersey.api.client.WebResource;
|
|
|
|
+import com.sun.jersey.api.client.filter.LoggingFilter;
|
|
import com.sun.jersey.api.json.JSONJAXBContext;
|
|
import com.sun.jersey.api.json.JSONJAXBContext;
|
|
import com.sun.jersey.api.json.JSONMarshaller;
|
|
import com.sun.jersey.api.json.JSONMarshaller;
|
|
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
|
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
|
@@ -461,11 +479,7 @@ public class TestRMWebServicesAppsModification extends JerseyTest {
|
|
.constructWebResource("apps", app.getApplicationId().toString(),
|
|
.constructWebResource("apps", app.getApplicationId().toString(),
|
|
"state").accept(mediaType)
|
|
"state").accept(mediaType)
|
|
.entity(info, MediaType.APPLICATION_XML).put(ClientResponse.class);
|
|
.entity(info, MediaType.APPLICATION_XML).put(ClientResponse.class);
|
|
- if (!isAuthenticationEnabled()) {
|
|
|
|
- assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
|
|
|
|
- } else {
|
|
|
|
- assertEquals(Status.FORBIDDEN, response.getClientResponseStatus());
|
|
|
|
- }
|
|
|
|
|
|
+ validateResponseStatus(response, Status.FORBIDDEN);
|
|
}
|
|
}
|
|
rm.stop();
|
|
rm.stop();
|
|
return;
|
|
return;
|
|
@@ -502,4 +516,348 @@ public class TestRMWebServicesAppsModification extends JerseyTest {
|
|
}
|
|
}
|
|
super.tearDown();
|
|
super.tearDown();
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Helper function to wrap frequently used code. It checks the response status
|
|
|
|
+ * and checks if it UNAUTHORIZED if we are running with authorization turned
|
|
|
|
+ * off or the param passed if we are running with authorization turned on.
|
|
|
|
+ *
|
|
|
|
+ * @param response
|
|
|
|
+ * the ClientResponse object to be checked
|
|
|
|
+ * @param expectedAuthorizedMode
|
|
|
|
+ * the expected Status in authorized mode.
|
|
|
|
+ */
|
|
|
|
+ public void validateResponseStatus(ClientResponse response,
|
|
|
|
+ Status expectedAuthorizedMode) {
|
|
|
|
+ validateResponseStatus(response, Status.UNAUTHORIZED,
|
|
|
|
+ expectedAuthorizedMode);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Helper function to wrap frequently used code. It checks the response status
|
|
|
|
+ * and checks if it is the param expectedUnauthorizedMode if we are running
|
|
|
|
+ * with authorization turned off or the param expectedAuthorizedMode passed if
|
|
|
|
+ * we are running with authorization turned on.
|
|
|
|
+ *
|
|
|
|
+ * @param response
|
|
|
|
+ * the ClientResponse object to be checked
|
|
|
|
+ * @param expectedUnauthorizedMode
|
|
|
|
+ * the expected Status in unauthorized mode.
|
|
|
|
+ * @param expectedAuthorizedMode
|
|
|
|
+ * the expected Status in authorized mode.
|
|
|
|
+ */
|
|
|
|
+ public void validateResponseStatus(ClientResponse response,
|
|
|
|
+ Status expectedUnauthorizedMode, Status expectedAuthorizedMode) {
|
|
|
|
+ if (!isAuthenticationEnabled()) {
|
|
|
|
+ assertEquals(expectedUnauthorizedMode, response.getClientResponseStatus());
|
|
|
|
+ } else {
|
|
|
|
+ assertEquals(expectedAuthorizedMode, response.getClientResponseStatus());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Simple test - just post to /apps/id and validate the response
|
|
|
|
+ @Test
|
|
|
|
+ public void testGetNewApplication() throws Exception {
|
|
|
|
+ // client().addFilter(new LoggingFilter(System.out));
|
|
|
|
+ rm.start();
|
|
|
|
+ String mediaTypes[] =
|
|
|
|
+ { MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML };
|
|
|
|
+ for (String acceptMedia : mediaTypes) {
|
|
|
|
+ testGetNewApplication(acceptMedia);
|
|
|
|
+ }
|
|
|
|
+ rm.stop();
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected String testGetNewApplication(String mediaType) throws JSONException,
|
|
|
|
+ ParserConfigurationException, IOException, SAXException {
|
|
|
|
+ ClientResponse response =
|
|
|
|
+ this.constructWebResource("apps", "new-application").accept(mediaType)
|
|
|
|
+ .post(ClientResponse.class);
|
|
|
|
+ validateResponseStatus(response, Status.OK);
|
|
|
|
+ if (!isAuthenticationEnabled()) {
|
|
|
|
+ return "";
|
|
|
|
+ }
|
|
|
|
+ return validateGetNewApplicationResponse(response);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected String validateGetNewApplicationResponse(ClientResponse resp)
|
|
|
|
+ throws JSONException, ParserConfigurationException, IOException,
|
|
|
|
+ SAXException {
|
|
|
|
+ String ret = "";
|
|
|
|
+ if (resp.getType().equals(MediaType.APPLICATION_JSON_TYPE)) {
|
|
|
|
+ JSONObject json = resp.getEntity(JSONObject.class);
|
|
|
|
+ ret = validateGetNewApplicationJsonResponse(json);
|
|
|
|
+ } else if (resp.getType().equals(MediaType.APPLICATION_XML_TYPE)) {
|
|
|
|
+ String xml = resp.getEntity(String.class);
|
|
|
|
+ ret = validateGetNewApplicationXMLResponse(xml);
|
|
|
|
+ } else {
|
|
|
|
+ // we should not be here
|
|
|
|
+ assertTrue(false);
|
|
|
|
+ }
|
|
|
|
+ return ret;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected String validateGetNewApplicationJsonResponse(JSONObject json)
|
|
|
|
+ throws JSONException {
|
|
|
|
+ String appId = json.getString("application-id");
|
|
|
|
+ assertTrue(appId.isEmpty() == false);
|
|
|
|
+ JSONObject maxResources = json.getJSONObject("maximum-resource-capability");
|
|
|
|
+ long memory = maxResources.getLong("memory");
|
|
|
|
+ long vCores = maxResources.getLong("vCores");
|
|
|
|
+ assertTrue(memory != 0);
|
|
|
|
+ assertTrue(vCores != 0);
|
|
|
|
+ return appId;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected String validateGetNewApplicationXMLResponse(String response)
|
|
|
|
+ throws ParserConfigurationException, IOException, SAXException {
|
|
|
|
+ DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
|
|
|
|
+ DocumentBuilder db = dbf.newDocumentBuilder();
|
|
|
|
+ InputSource is = new InputSource();
|
|
|
|
+ is.setCharacterStream(new StringReader(response));
|
|
|
|
+ Document dom = db.parse(is);
|
|
|
|
+ NodeList nodes = dom.getElementsByTagName("NewApplication");
|
|
|
|
+ assertEquals("incorrect number of elements", 1, nodes.getLength());
|
|
|
|
+ Element element = (Element) nodes.item(0);
|
|
|
|
+ String appId = WebServicesTestUtils.getXmlString(element, "application-id");
|
|
|
|
+ assertTrue(appId.isEmpty() == false);
|
|
|
|
+ NodeList maxResourceNodes =
|
|
|
|
+ element.getElementsByTagName("maximum-resource-capability");
|
|
|
|
+ assertEquals(1, maxResourceNodes.getLength());
|
|
|
|
+ Element maxResourceCapability = (Element) maxResourceNodes.item(0);
|
|
|
|
+ long memory =
|
|
|
|
+ WebServicesTestUtils.getXmlLong(maxResourceCapability, "memory");
|
|
|
|
+ long vCores =
|
|
|
|
+ WebServicesTestUtils.getXmlLong(maxResourceCapability, "vCores");
|
|
|
|
+ assertTrue(memory != 0);
|
|
|
|
+ assertTrue(vCores != 0);
|
|
|
|
+ return appId;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Test to validate the process of submitting apps - test for appropriate
|
|
|
|
+ // errors as well
|
|
|
|
+ @Test
|
|
|
|
+ public void testGetNewApplicationAndSubmit() throws Exception {
|
|
|
|
+ rm.start();
|
|
|
|
+ MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
|
|
|
|
+ amNodeManager.nodeHeartbeat(true);
|
|
|
|
+ String mediaTypes[] =
|
|
|
|
+ { MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML };
|
|
|
|
+ for (String acceptMedia : mediaTypes) {
|
|
|
|
+ for (String contentMedia : mediaTypes) {
|
|
|
|
+ testAppSubmit(acceptMedia, contentMedia);
|
|
|
|
+ testAppSubmitErrors(acceptMedia, contentMedia);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ rm.stop();
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void testAppSubmit(String acceptMedia, String contentMedia)
|
|
|
|
+ throws Exception {
|
|
|
|
+
|
|
|
|
+ // create a test app and submit it via rest(after getting an app-id) then
|
|
|
|
+ // get the app details from the rmcontext and check that everything matches
|
|
|
|
+
|
|
|
|
+ // client().addFilter(new LoggingFilter(System.out));
|
|
|
|
+ String lrKey = "example";
|
|
|
|
+ String queueName = "testqueue";
|
|
|
|
+ String appName = "test";
|
|
|
|
+ String appType = "test-type";
|
|
|
|
+ String urlPath = "apps";
|
|
|
|
+ String appId = testGetNewApplication(acceptMedia);
|
|
|
|
+ List<String> commands = new ArrayList<String>();
|
|
|
|
+ commands.add("/bin/sleep 5");
|
|
|
|
+ HashMap<String, String> environment = new HashMap<String, String>();
|
|
|
|
+ environment.put("APP_VAR", "ENV_SETTING");
|
|
|
|
+ HashMap<ApplicationAccessType, String> acls =
|
|
|
|
+ new HashMap<ApplicationAccessType, String>();
|
|
|
|
+ acls.put(ApplicationAccessType.MODIFY_APP, "testuser1, testuser2");
|
|
|
|
+ acls.put(ApplicationAccessType.VIEW_APP, "testuser3, testuser4");
|
|
|
|
+ Set<String> tags = new HashSet<String>();
|
|
|
|
+ tags.add("tag1");
|
|
|
|
+ tags.add("tag 2");
|
|
|
|
+ CredentialsInfo credentials = new CredentialsInfo();
|
|
|
|
+ HashMap<String, String> tokens = new HashMap<String, String>();
|
|
|
|
+ HashMap<String, String> secrets = new HashMap<String, String>();
|
|
|
|
+ secrets.put("secret1", Base64.encodeBase64URLSafeString("secret1".getBytes("UTF8")));
|
|
|
|
+ credentials.setSecrets(secrets);
|
|
|
|
+ credentials.setTokens(tokens);
|
|
|
|
+ ApplicationSubmissionContextInfo appInfo = new ApplicationSubmissionContextInfo();
|
|
|
|
+ appInfo.setApplicationId(appId);
|
|
|
|
+ appInfo.setApplicationName(appName);
|
|
|
|
+ appInfo.setPriority(3);
|
|
|
|
+ appInfo.setMaxAppAttempts(2);
|
|
|
|
+ appInfo.setQueue(queueName);
|
|
|
|
+ appInfo.setApplicationType(appType);
|
|
|
|
+ HashMap<String, LocalResourceInfo> lr =
|
|
|
|
+ new HashMap<String, LocalResourceInfo>();
|
|
|
|
+ LocalResourceInfo y = new LocalResourceInfo();
|
|
|
|
+ y.setUrl(new URI("http://www.test.com/file.txt"));
|
|
|
|
+ y.setSize(100);
|
|
|
|
+ y.setTimestamp(System.currentTimeMillis());
|
|
|
|
+ y.setType(LocalResourceType.FILE);
|
|
|
|
+ y.setVisibility(LocalResourceVisibility.APPLICATION);
|
|
|
|
+ lr.put(lrKey, y);
|
|
|
|
+ appInfo.getContainerLaunchContextInfo().setResources(lr);
|
|
|
|
+ appInfo.getContainerLaunchContextInfo().setCommands(commands);
|
|
|
|
+ appInfo.getContainerLaunchContextInfo().setEnvironment(environment);
|
|
|
|
+ appInfo.getContainerLaunchContextInfo().setAcls(acls);
|
|
|
|
+ appInfo.getContainerLaunchContextInfo().getAuxillaryServiceData()
|
|
|
|
+ .put("test", Base64.encodeBase64URLSafeString("value12".getBytes("UTF8")));
|
|
|
|
+ appInfo.getContainerLaunchContextInfo().setCredentials(credentials);
|
|
|
|
+ appInfo.getResource().setMemory(1024);
|
|
|
|
+ appInfo.getResource().setvCores(1);
|
|
|
|
+ appInfo.setApplicationTags(tags);
|
|
|
|
+
|
|
|
|
+ ClientResponse response =
|
|
|
|
+ this.constructWebResource(urlPath).accept(acceptMedia)
|
|
|
|
+ .entity(appInfo, contentMedia).post(ClientResponse.class);
|
|
|
|
+
|
|
|
|
+ if (this.isAuthenticationEnabled() == false) {
|
|
|
|
+ assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ assertEquals(Status.ACCEPTED, response.getClientResponseStatus());
|
|
|
|
+ assertTrue(response.getHeaders().getFirst(HttpHeaders.LOCATION).isEmpty() == false);
|
|
|
|
+ String locURL = response.getHeaders().getFirst(HttpHeaders.LOCATION);
|
|
|
|
+ assertTrue(locURL.indexOf("/apps/application") != -1);
|
|
|
|
+ appId = locURL.substring(locURL.indexOf("/apps/") + "/apps/".length());
|
|
|
|
+
|
|
|
|
+ WebResource res = resource().uri(new URI(locURL));
|
|
|
|
+ res = res.queryParam("user.name", webserviceUserName);
|
|
|
|
+ response = res.get(ClientResponse.class);
|
|
|
|
+ assertEquals(Status.OK, response.getClientResponseStatus());
|
|
|
|
+
|
|
|
|
+ RMApp app =
|
|
|
|
+ rm.getRMContext().getRMApps()
|
|
|
|
+ .get(ConverterUtils.toApplicationId(appId));
|
|
|
|
+ assertEquals(appName, app.getName());
|
|
|
|
+ assertEquals(webserviceUserName, app.getUser());
|
|
|
|
+ assertEquals(2, app.getMaxAppAttempts());
|
|
|
|
+ assertEquals(queueName, app.getQueue());
|
|
|
|
+ assertEquals(appType, app.getApplicationType());
|
|
|
|
+ assertEquals(tags, app.getApplicationTags());
|
|
|
|
+ ContainerLaunchContext ctx =
|
|
|
|
+ app.getApplicationSubmissionContext().getAMContainerSpec();
|
|
|
|
+ assertEquals(commands, ctx.getCommands());
|
|
|
|
+ assertEquals(environment, ctx.getEnvironment());
|
|
|
|
+ assertEquals(acls, ctx.getApplicationACLs());
|
|
|
|
+ Map<String, LocalResource> appLRs = ctx.getLocalResources();
|
|
|
|
+ assertTrue(appLRs.containsKey(lrKey));
|
|
|
|
+ LocalResource exampleLR = appLRs.get(lrKey);
|
|
|
|
+ assertEquals(ConverterUtils.getYarnUrlFromURI(y.getUrl()),
|
|
|
|
+ exampleLR.getResource());
|
|
|
|
+ assertEquals(y.getSize(), exampleLR.getSize());
|
|
|
|
+ assertEquals(y.getTimestamp(), exampleLR.getTimestamp());
|
|
|
|
+ assertEquals(y.getType(), exampleLR.getType());
|
|
|
|
+ assertEquals(y.getPattern(), exampleLR.getPattern());
|
|
|
|
+ assertEquals(y.getVisibility(), exampleLR.getVisibility());
|
|
|
|
+
|
|
|
|
+ response =
|
|
|
|
+ this.constructWebResource("apps", appId).accept(acceptMedia)
|
|
|
|
+ .get(ClientResponse.class);
|
|
|
|
+ assertEquals(Status.OK, response.getClientResponseStatus());
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void testAppSubmitErrors(String acceptMedia, String contentMedia)
|
|
|
|
+ throws Exception {
|
|
|
|
+
|
|
|
|
+ // submit a bunch of bad requests(correct format but bad values) via the
|
|
|
|
+ // REST API and make sure we get the right error response codes
|
|
|
|
+
|
|
|
|
+ String urlPath = "apps";
|
|
|
|
+ String appId = "";
|
|
|
|
+ ApplicationSubmissionContextInfo appInfo = new ApplicationSubmissionContextInfo();
|
|
|
|
+ ClientResponse response =
|
|
|
|
+ this.constructWebResource(urlPath).accept(acceptMedia)
|
|
|
|
+ .entity(appInfo, contentMedia).post(ClientResponse.class);
|
|
|
|
+ validateResponseStatus(response, Status.BAD_REQUEST);
|
|
|
|
+
|
|
|
|
+ appId = "random";
|
|
|
|
+ appInfo.setApplicationId(appId);
|
|
|
|
+ response =
|
|
|
|
+ this.constructWebResource(urlPath).accept(acceptMedia)
|
|
|
|
+ .entity(appInfo, contentMedia).post(ClientResponse.class);
|
|
|
|
+ validateResponseStatus(response, Status.BAD_REQUEST);
|
|
|
|
+
|
|
|
|
+ appId = "random_junk";
|
|
|
|
+ appInfo.setApplicationId(appId);
|
|
|
|
+ response =
|
|
|
|
+ this.constructWebResource(urlPath).accept(acceptMedia)
|
|
|
|
+ .entity(appInfo, contentMedia).post(ClientResponse.class);
|
|
|
|
+ validateResponseStatus(response, Status.BAD_REQUEST);
|
|
|
|
+
|
|
|
|
+ // bad resource info
|
|
|
|
+ appInfo.getResource().setMemory(
|
|
|
|
+ rm.getConfig().getInt(
|
|
|
|
+ YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB) + 1);
|
|
|
|
+ appInfo.getResource().setvCores(1);
|
|
|
|
+ response =
|
|
|
|
+ this.constructWebResource(urlPath).accept(acceptMedia)
|
|
|
|
+ .entity(appInfo, contentMedia).post(ClientResponse.class);
|
|
|
|
+
|
|
|
|
+ validateResponseStatus(response, Status.BAD_REQUEST);
|
|
|
|
+
|
|
|
|
+ appInfo.getResource().setvCores(
|
|
|
|
+ rm.getConfig().getInt(
|
|
|
|
+ YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
|
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES) + 1);
|
|
|
|
+ appInfo.getResource().setMemory(CONTAINER_MB);
|
|
|
|
+ response =
|
|
|
|
+ this.constructWebResource(urlPath).accept(acceptMedia)
|
|
|
|
+ .entity(appInfo, contentMedia).post(ClientResponse.class);
|
|
|
|
+ validateResponseStatus(response, Status.BAD_REQUEST);
|
|
|
|
+
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testAppSubmitBadJsonAndXML() throws Exception {
|
|
|
|
+
|
|
|
|
+ // submit a bunch of bad XML and JSON via the
|
|
|
|
+ // REST API and make sure we get error response codes
|
|
|
|
+
|
|
|
|
+ String urlPath = "apps";
|
|
|
|
+ rm.start();
|
|
|
|
+ MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
|
|
|
|
+ amNodeManager.nodeHeartbeat(true);
|
|
|
|
+
|
|
|
|
+ ApplicationSubmissionContextInfo appInfo = new ApplicationSubmissionContextInfo();
|
|
|
|
+ appInfo.setApplicationName("test");
|
|
|
|
+ appInfo.setPriority(3);
|
|
|
|
+ appInfo.setMaxAppAttempts(2);
|
|
|
|
+ appInfo.setQueue("testqueue");
|
|
|
|
+ appInfo.setApplicationType("test-type");
|
|
|
|
+ HashMap<String, LocalResourceInfo> lr =
|
|
|
|
+ new HashMap<String, LocalResourceInfo>();
|
|
|
|
+ LocalResourceInfo y = new LocalResourceInfo();
|
|
|
|
+ y.setUrl(new URI("http://www.test.com/file.txt"));
|
|
|
|
+ y.setSize(100);
|
|
|
|
+ y.setTimestamp(System.currentTimeMillis());
|
|
|
|
+ y.setType(LocalResourceType.FILE);
|
|
|
|
+ y.setVisibility(LocalResourceVisibility.APPLICATION);
|
|
|
|
+ lr.put("example", y);
|
|
|
|
+ appInfo.getContainerLaunchContextInfo().setResources(lr);
|
|
|
|
+ appInfo.getResource().setMemory(1024);
|
|
|
|
+ appInfo.getResource().setvCores(1);
|
|
|
|
+
|
|
|
|
+ String body =
|
|
|
|
+ "<?xml version=\"1.0\" encoding=\"UTF-8\" "
|
|
|
|
+ + "standalone=\"yes\"?><blah/>";
|
|
|
|
+ ClientResponse response =
|
|
|
|
+ this.constructWebResource(urlPath).accept(MediaType.APPLICATION_XML)
|
|
|
|
+ .entity(body, MediaType.APPLICATION_XML).post(ClientResponse.class);
|
|
|
|
+ assertEquals(Status.BAD_REQUEST, response.getClientResponseStatus());
|
|
|
|
+ body = "{\"a\" : \"b\"}";
|
|
|
|
+ response =
|
|
|
|
+ this.constructWebResource(urlPath).accept(MediaType.APPLICATION_XML)
|
|
|
|
+ .entity(body, MediaType.APPLICATION_JSON).post(ClientResponse.class);
|
|
|
|
+ validateResponseStatus(response, Status.BAD_REQUEST);
|
|
|
|
+ rm.stop();
|
|
|
|
+ }
|
|
|
|
+
|
|
}
|
|
}
|