diff --git a/LICENSE.txt b/LICENSE.txt index 3f50521..46ee108 100644 --- a/LICENSE.txt +++ b/LICENSE.txt @@ -1776,6 +1776,7 @@ SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. The binary distribution of this product bundles these dependencies under the following license: FindBugs-jsr305 3.0.0 +dnsjava 2.1.7, Copyright (c) 1998-2011, Brian Wellington. All rights reserved. -------------------------------------------------------------------------------- (2-clause BSD) Redistribution and use in source and binary forms, with or without diff --git a/NOTICE.txt b/NOTICE.txt index 0718909..f3af2f7 100644 --- a/NOTICE.txt +++ b/NOTICE.txt @@ -581,3 +581,17 @@ The binary distribution of this product bundles binaries of Ehcache 3.3.1, which has the following notices: * Ehcache V3 Copyright 2014-2016 Terracotta, Inc. + +JCommander (https://github.com/cbeust/jcommander), +which has the following notices: + * Copyright 2010 Cedric Beust cedric@beust.com + +The binary distribution of this product bundles binaries of +snakeyaml (https://bitbucket.org/asomov/snakeyaml), +which has the following notices: + * Copyright (c) 2008, http://www.snakeyaml.org + +The binary distribution of this product bundles binaries of +swagger-annotations (https://github.com/swagger-api/swagger-core), +which has the following notices: + * Copyright 2016 SmartBear Software diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml index 289061f..83633ac 100644 --- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml +++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml @@ -87,6 +87,31 @@ + hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/target + /share/hadoop/${hadoop.component}/sources + + *-sources.jar + + + + hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/conf + etc/hadoop + + + hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples + /share/hadoop/${hadoop.component}/yarn-service-examples + + **/* + + + + hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/target + /share/hadoop/${hadoop.component}/sources + + *-sources.jar + + + hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/target /share/hadoop/${hadoop.component}/sources diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 8980f0e..7e45e1a 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -89,6 +89,7 @@ 2.12.0 3.0.0 3.1.0-RC1 + 2.1.7 11.0.2 4.0 @@ -141,6 +142,10 @@ ${project.version} + + 1.5.4 + 1.16 + 1.48 @@ -423,6 +428,12 @@ org.apache.hadoop + hadoop-yarn-services-core + ${project.version} + + + + org.apache.hadoop hadoop-mapreduce-client-jobclient ${project.version} test-jar @@ -597,6 +608,11 @@ 3.1.0 + javax.ws.rs + jsr311-api + 1.1.1 + + org.eclipse.jetty jetty-server ${jetty.version} @@ -951,11 +967,6 @@ ${jackson2.version} - com.fasterxml.jackson.jaxrs - jackson-jaxrs-json-provider - ${jackson2.version} - - com.fasterxml.jackson.dataformat jackson-dataformat-cbor ${jackson2.version} @@ -1207,6 +1218,13 @@ + + + dnsjava + dnsjava + ${dnsjava.version} + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml new file mode 100644 index 0000000..ddea2a1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml @@ -0,0 +1,130 @@ + + + 4.0.0 + + org.apache.hadoop + hadoop-yarn-applications + 3.1.0-SNAPSHOT + + hadoop-yarn-services-api + Apache Hadoop YARN Services API + jar + Hadoop YARN REST APIs for services + + + + + + + src/main/resources + true + + + src/main/scripts/ + true + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + + + development + ${project.url} + + + + + + + + + + test-jar + + + + + + + + + + + + + + org.apache.hadoop + hadoop-yarn-services-core + + + org.apache.hadoop + hadoop-yarn-api + + + org.apache.hadoop + hadoop-yarn-common + + + org.apache.hadoop + hadoop-common + + + org.slf4j + slf4j-api + + + org.eclipse.jetty + jetty-webapp + + + com.google.inject + guice + + + javax.ws.rs + jsr311-api + + + org.mockito + mockito-all + test + + + + + + + + org.apache.hadoop + hadoop-common + test-jar + + + junit + junit + test + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java new file mode 100644 index 0000000..5773069 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java @@ -0,0 +1,307 @@ +/* + * 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.service.webapp; + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.VersionInfo; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.api.records.Service; +import org.apache.hadoop.yarn.service.api.records.ServiceState; +import org.apache.hadoop.yarn.service.api.records.ServiceStatus; +import org.apache.hadoop.yarn.service.client.ServiceClient; +import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +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.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.Status; +import java.io.IOException; +import java.util.Collections; +import java.util.Map; + +import static org.apache.hadoop.yarn.service.api.records.ServiceState.ACCEPTED; +import static org.apache.hadoop.yarn.service.conf.RestApiConstants.*; + +/** + * The rest API endpoints for users to manage services on YARN. + */ +@Singleton +@Path(CONTEXT_ROOT) +public class ApiServer { + + public ApiServer() { + super(); + } + + @Inject + public ApiServer(Configuration conf) { + super(); + } + + private static final Logger LOG = + LoggerFactory.getLogger(ApiServer.class); + private static Configuration YARN_CONFIG = new YarnConfiguration(); + private static ServiceClient SERVICE_CLIENT; + + static { + init(); + } + + // initialize all the common resources - order is important + private static void init() { + SERVICE_CLIENT = new ServiceClient(); + SERVICE_CLIENT.init(YARN_CONFIG); + SERVICE_CLIENT.start(); + } + + @GET + @Path(VERSION) + @Consumes({ MediaType.APPLICATION_JSON }) + @Produces({ MediaType.APPLICATION_JSON }) + public Response getVersion() { + String version = VersionInfo.getBuildVersion(); + LOG.info(version); + return Response.ok("{ \"hadoop_version\": \"" + version + "\"}").build(); + } + + @POST + @Path(SERVICE_ROOT_PATH) + @Consumes({ MediaType.APPLICATION_JSON }) + @Produces({ MediaType.APPLICATION_JSON }) + public Response createService(Service service) { + LOG.info("POST: createService = {}", service); + ServiceStatus serviceStatus = new ServiceStatus(); + try { + ApplicationId applicationId = SERVICE_CLIENT.actionCreate(service); + LOG.info("Successfully created service " + service.getName() + + " applicationId = " + applicationId); + serviceStatus.setState(ACCEPTED); + serviceStatus.setUri( + CONTEXT_ROOT + SERVICE_ROOT_PATH + "/" + service + .getName()); + return Response.status(Status.ACCEPTED).entity(serviceStatus).build(); + } catch (IllegalArgumentException e) { + serviceStatus.setDiagnostics(e.getMessage()); + return Response.status(Status.BAD_REQUEST).entity(serviceStatus) + .build(); + } catch (Exception e) { + String message = "Failed to create service " + service.getName(); + LOG.error(message, e); + serviceStatus.setDiagnostics(message + ": " + e.getMessage()); + return Response.status(Status.INTERNAL_SERVER_ERROR) + .entity(serviceStatus).build(); + } + } + + @GET + @Path(SERVICE_PATH) + @Consumes({ MediaType.APPLICATION_JSON }) + @Produces({ MediaType.APPLICATION_JSON }) + public Response getService(@PathParam(SERVICE_NAME) String appName) { + LOG.info("GET: getService for appName = {}", appName); + ServiceStatus serviceStatus = new ServiceStatus(); + try { + Service app = SERVICE_CLIENT.getStatus(appName); + return Response.ok(app).build(); + } catch (IllegalArgumentException e) { + serviceStatus.setDiagnostics(e.getMessage()); + serviceStatus.setCode(ERROR_CODE_APP_NAME_INVALID); + return Response.status(Status.NOT_FOUND).entity(serviceStatus) + .build(); + } catch (Exception e) { + LOG.error("Get service failed", e); + serviceStatus + .setDiagnostics("Failed to retrieve service: " + e.getMessage()); + return Response.status(Status.INTERNAL_SERVER_ERROR) + .entity(serviceStatus).build(); + } + } + + @DELETE + @Path(SERVICE_PATH) + @Consumes({ MediaType.APPLICATION_JSON }) + @Produces({ MediaType.APPLICATION_JSON }) + public Response deleteService(@PathParam(SERVICE_NAME) String appName) { + LOG.info("DELETE: deleteService for appName = {}", appName); + return stopService(appName, true); + } + + private Response stopService(String appName, boolean destroy) { + try { + SERVICE_CLIENT.actionStop(appName, destroy); + if (destroy) { + SERVICE_CLIENT.actionDestroy(appName); + LOG.info("Successfully deleted service {}", appName); + } else { + LOG.info("Successfully stopped service {}", appName); + } + return Response.status(Status.OK).build(); + } catch (ApplicationNotFoundException e) { + ServiceStatus serviceStatus = new ServiceStatus(); + serviceStatus.setDiagnostics( + "Service " + appName + " is not found in YARN: " + e.getMessage()); + return Response.status(Status.BAD_REQUEST).entity(serviceStatus) + .build(); + } catch (Exception e) { + ServiceStatus serviceStatus = new ServiceStatus(); + serviceStatus.setDiagnostics(e.getMessage()); + return Response.status(Status.INTERNAL_SERVER_ERROR) + .entity(serviceStatus).build(); + } + } + + @PUT + @Path(COMPONENT_PATH) + @Consumes({ MediaType.APPLICATION_JSON }) + @Produces({ MediaType.APPLICATION_JSON, MediaType.TEXT_PLAIN }) + public Response updateComponent(@PathParam(SERVICE_NAME) String appName, + @PathParam(COMPONENT_NAME) String componentName, Component component) { + + if (component.getNumberOfContainers() < 0) { + return Response.status(Status.BAD_REQUEST).entity( + "Service = " + appName + ", Component = " + component.getName() + + ": Invalid number of containers specified " + component + .getNumberOfContainers()).build(); + } + ServiceStatus status = new ServiceStatus(); + try { + Map original = SERVICE_CLIENT.flexByRestService(appName, + Collections.singletonMap(component.getName(), + component.getNumberOfContainers())); + status.setDiagnostics( + "Updating component (" + componentName + ") size from " + original + .get(componentName) + " to " + component.getNumberOfContainers()); + return Response.ok().entity(status).build(); + } catch (YarnException | IOException e) { + status.setDiagnostics(e.getMessage()); + return Response.status(Status.INTERNAL_SERVER_ERROR).entity(status) + .build(); + } + } + + @PUT + @Path(SERVICE_PATH) + @Consumes({ MediaType.APPLICATION_JSON }) + @Produces({ MediaType.APPLICATION_JSON }) + public Response updateService(@PathParam(SERVICE_NAME) String appName, + Service updateServiceData) { + LOG.info("PUT: updateService for app = {} with data = {}", appName, + updateServiceData); + + // Ignore the app name provided in updateServiceData and always use appName + // path param + updateServiceData.setName(appName); + + // For STOP the app should be running. If already stopped then this + // operation will be a no-op. For START it should be in stopped state. + // If already running then this operation will be a no-op. + if (updateServiceData.getState() != null + && updateServiceData.getState() == ServiceState.STOPPED) { + return stopService(appName, false); + } + + // If a START is requested + if (updateServiceData.getState() != null + && updateServiceData.getState() == ServiceState.STARTED) { + return startService(appName); + } + + // If new lifetime value specified then update it + if (updateServiceData.getLifetime() != null + && updateServiceData.getLifetime() > 0) { + return updateLifetime(appName, updateServiceData); + } + + // flex a single component app + if (updateServiceData.getNumberOfContainers() != null && !ServiceApiUtil + .hasComponent(updateServiceData)) { + Component defaultComp = ServiceApiUtil + .createDefaultComponent(updateServiceData); + return updateComponent(updateServiceData.getName(), defaultComp.getName(), + defaultComp); + } + + // If nothing happens consider it a no-op + return Response.status(Status.NO_CONTENT).build(); + } + + private Response updateLifetime(String appName, Service updateAppData) { + ServiceStatus status = new ServiceStatus(); + try { + String newLifeTime = + SERVICE_CLIENT.updateLifetime(appName, updateAppData.getLifetime()); + status.setDiagnostics( + "Service (" + appName + ")'s lifeTime is updated to " + newLifeTime + + ", " + updateAppData.getLifetime() + + " seconds remaining"); + return Response.ok(status).build(); + } catch (Exception e) { + String message = + "Failed to update service (" + appName + ")'s lifetime to " + + updateAppData.getLifetime(); + LOG.error(message, e); + status.setDiagnostics(message + ": " + e.getMessage()); + return Response.status(Status.INTERNAL_SERVER_ERROR).entity(status) + .build(); + } + } + + private Response startService(String appName) { + ServiceStatus status = new ServiceStatus(); + try { + SERVICE_CLIENT.actionStart(appName); + LOG.info("Successfully started service " + appName); + status.setDiagnostics("Service " + appName + " is successfully started."); + status.setState(ServiceState.ACCEPTED); + return Response.ok(status).build(); + } catch (Exception e) { + String message = "Failed to start service " + appName; + status.setDiagnostics(message + ": " + e.getMessage()); + LOG.info(message, e); + return Response.status(Status.INTERNAL_SERVER_ERROR) + .entity(status).build(); + } + } + + /** + * Used by negative test case. + * + * @param mockServerClient - A mocked version of ServiceClient + */ + public static void setServiceClient(ServiceClient mockServerClient) { + SERVICE_CLIENT = mockServerClient; + SERVICE_CLIENT.init(YARN_CONFIG); + SERVICE_CLIENT.start(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServerWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServerWebApp.java new file mode 100644 index 0000000..f4acd94 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServerWebApp.java @@ -0,0 +1,161 @@ +/* + * 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.service.webapp; + +import org.apache.hadoop.http.HttpServer2; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.AuthenticationFilterInitializer; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; +import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider; +import org.eclipse.jetty.webapp.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.URI; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.apache.hadoop.yarn.conf.YarnConfiguration.RM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY; +import static org.apache.hadoop.yarn.conf.YarnConfiguration.RM_WEBAPP_SPNEGO_USER_NAME_KEY; +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.*; + +/** + * This class launches the web service using Hadoop HttpServer2 (which uses + * an embedded Jetty container). This is the entry point to your service. + * The Java command used to launch this app should call the main method. + */ +public class ApiServerWebApp extends AbstractService { + private static final Logger logger = LoggerFactory + .getLogger(ApiServerWebApp.class); + private static final String SEP = ";"; + + // REST API server for YARN native services + private HttpServer2 apiServer; + private InetSocketAddress bindAddress; + + public static void main(String[] args) throws IOException { + ApiServerWebApp apiWebApp = new ApiServerWebApp(); + try { + apiWebApp.init(new YarnConfiguration()); + apiWebApp.serviceStart(); + } catch (Exception e) { + logger.error("Got exception starting", e); + apiWebApp.close(); + } + } + + public ApiServerWebApp() { + super(ApiServerWebApp.class.getName()); + } + + @Override + protected void serviceStart() throws Exception { + bindAddress = getConfig().getSocketAddr(API_SERVER_ADDRESS, + DEFAULT_API_SERVER_ADDRESS, DEFAULT_API_SERVER_PORT); + logger.info("YARN API server running on " + bindAddress); + if (UserGroupInformation.isSecurityEnabled()) { + doSecureLogin(getConfig()); + } + startWebApp(); + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + if (apiServer != null) { + apiServer.stop(); + } + super.serviceStop(); + } + + private void doSecureLogin(org.apache.hadoop.conf.Configuration conf) + throws IOException { + SecurityUtil.login(conf, YarnConfiguration.RM_KEYTAB, + YarnConfiguration.RM_PRINCIPAL, bindAddress.getHostName()); + addFilters(conf); + } + + private void addFilters(org.apache.hadoop.conf.Configuration conf) { + // Always load pseudo authentication filter to parse "user.name" in an URL + // to identify a HTTP request's user. + boolean hasHadoopAuthFilterInitializer = false; + String filterInitializerConfKey = "hadoop.http.filter.initializers"; + Class[] initializersClasses = + conf.getClasses(filterInitializerConfKey); + List targets = new ArrayList(); + if (initializersClasses != null) { + for (Class initializer : initializersClasses) { + if (initializer.getName().equals( + AuthenticationFilterInitializer.class.getName())) { + hasHadoopAuthFilterInitializer = true; + break; + } + targets.add(initializer.getName()); + } + } + if (!hasHadoopAuthFilterInitializer) { + targets.add(AuthenticationFilterInitializer.class.getName()); + conf.set(filterInitializerConfKey, StringUtils.join(",", targets)); + } + } + + private void startWebApp() throws IOException { + URI uri = URI.create("http://" + NetUtils.getHostPortString(bindAddress)); + + apiServer = new HttpServer2.Builder() + .setName("api-server") + .setConf(getConfig()) + .setSecurityEnabled(UserGroupInformation.isSecurityEnabled()) + .setUsernameConfKey(RM_WEBAPP_SPNEGO_USER_NAME_KEY) + .setKeytabConfKey(RM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY) + .addEndpoint(uri).build(); + + String apiPackages = + ApiServer.class.getPackage().getName() + SEP + + GenericExceptionHandler.class.getPackage().getName() + SEP + + YarnJacksonJaxbJsonProvider.class.getPackage().getName(); + apiServer.addJerseyResourcePackage(apiPackages, "/*"); + + try { + logger.info("Service starting up. Logging start..."); + apiServer.start(); + logger.info("Server status = {}", apiServer.toString()); + for (Configuration conf : apiServer.getWebAppContext() + .getConfigurations()) { + logger.info("Configurations = {}", conf); + } + logger.info("Context Path = {}", Collections.singletonList( + apiServer.getWebAppContext().getContextPath())); + logger.info("ResourceBase = {}", Collections.singletonList( + apiServer.getWebAppContext().getResourceBase())); + logger.info("War = {}", Collections + .singletonList(apiServer.getWebAppContext().getWar())); + } catch (Exception ex) { + logger.error("Hadoop HttpServer2 App **failed**", ex); + throw ex; + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md new file mode 100644 index 0000000..3cd3d48 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md @@ -0,0 +1,245 @@ + + +## Examples + +### Create a simple single-component service with most attribute values as defaults +POST URL - http://localhost:9191/ws/v1/services + +##### POST Request JSON +```json +{ + "name": "hello-world", + "components" : + [ + { + "name": "hello", + "number_of_containers": 1, + "artifact": { + "id": "nginx:latest", + "type": "DOCKER" + }, + "launch_command": "./start_nginx.sh", + "resource": { + "cpus": 1, + "memory": "256" + } + } + ] +} +``` + +##### GET Response JSON +GET URL - http://localhost:9191/ws/v1/services/hello-world + +Note, lifetime value of -1 means unlimited lifetime. + +```json +{ + "name": "hello-world", + "id": "application_1503963985568_0002", + "lifetime": -1, + "components": [ + { + "name": "hello", + "dependencies": [], + "resource": { + "cpus": 1, + "memory": "256" + }, + "configuration": { + "properties": {}, + "env": {}, + "files": [] + }, + "quicklinks": [], + "containers": [ + { + "id": "container_e03_1503963985568_0002_01_000001", + "ip": "10.22.8.143", + "hostname": "myhost.local", + "state": "READY", + "launch_time": 1504051512412, + "bare_host": "10.22.8.143", + "component_name": "hello-0" + }, + { + "id": "container_e03_1503963985568_0002_01_000002", + "ip": "10.22.8.143", + "hostname": "myhost.local", + "state": "READY", + "launch_time": 1504051536450, + "bare_host": "10.22.8.143", + "component_name": "hello-1" + } + ], + "launch_command": "./start_nginx.sh", + "number_of_containers": 1, + "run_privileged_container": false + } + ], + "configuration": { + "properties": {}, + "env": {}, + "files": [] + }, + "quicklinks": {} +} + +``` +### Update to modify the lifetime of a service +PUT URL - http://localhost:9191/ws/v1/services/hello-world + +##### PUT Request JSON + +Note, irrespective of what the current lifetime value is, this update request will set the lifetime of the service to be 3600 seconds (1 hour) from the time the request is submitted. Hence, if a a service has remaining lifetime of 5 mins (say) and would like to extend it to an hour OR if an application has remaining lifetime of 5 hours (say) and would like to reduce it down to an hour, then for both scenarios you need to submit the same request below. + +```json +{ + "lifetime": 3600 +} +``` +### Stop a service +PUT URL - http://localhost:9191/ws/v1/services/hello-world + +##### PUT Request JSON +```json +{ + "state": "STOPPED" +} +``` + +### Start a service +PUT URL - http://localhost:9191/ws/v1/services/hello-world + +##### PUT Request JSON +```json +{ + "state": "STARTED" +} +``` + +### Update to flex up/down the no of containers (instances) of a component of a service +PUT URL - http://localhost:9191/ws/v1/services/hello-world/components/hello + +##### PUT Request JSON +```json +{ + "name": "hello", + "number_of_containers": 3 +} +``` + +### Destroy a service +DELETE URL - http://localhost:9191/ws/v1/services/hello-world + +*** + +### Create a complicated service - HBase +POST URL - http://localhost:9191:/ws/v1/services/hbase-app-1 + +##### POST Request JSON + +```json +{ + "name": "hbase-app-1", + "lifetime": "3600", + "components": [ + { + "name": "hbasemaster", + "number_of_containers": 1, + "artifact": { + "id": "hbase:latest", + "type": "DOCKER" + }, + "launch_command": "/usr/hdp/current/hbase-master/bin/hbase master start", + "resource": { + "cpus": 1, + "memory": "2048" + }, + "configuration": { + "env": { + "HBASE_LOG_DIR": "" + }, + "files": [ + { + "type": "XML", + "dest_file": "/etc/hadoop/conf/core-site.xml", + "props": { + "fs.defaultFS": "${CLUSTER_FS_URI}" + } + }, + { + "type": "XML", + "dest_file": "/etc/hbase/conf/hbase-site.xml", + "props": { + "hbase.cluster.distributed": "true", + "hbase.zookeeper.quorum": "${CLUSTER_ZK_QUORUM}", + "hbase.rootdir": "${SERVICE_HDFS_DIR}/hbase", + "zookeeper.znode.parent": "${SERVICE_ZK_PATH}", + "hbase.master.hostname": "hbasemaster.${SERVICE_NAME}.${USER}.${DOMAIN}", + "hbase.master.info.port": "16010" + } + } + ] + } + }, + { + "name": "regionserver", + "number_of_containers": 3, + "unique_component_support": "true", + "artifact": { + "id": "hbase:latest", + "type": "DOCKER" + }, + "launch_command": "/usr/hdp/current/hbase-regionserver/bin/hbase regionserver start", + "resource": { + "cpus": 1, + "memory": "2048" + }, + "configuration": { + "env": { + "HBASE_LOG_DIR": "" + }, + "files": [ + { + "type": "XML", + "dest_file": "/etc/hadoop/conf/core-site.xml", + "props": { + "fs.defaultFS": "${CLUSTER_FS_URI}" + } + }, + { + "type": "XML", + "dest_file": "/etc/hbase/conf/hbase-site.xml", + "props": { + "hbase.cluster.distributed": "true", + "hbase.zookeeper.quorum": "${CLUSTER_ZK_QUORUM}", + "hbase.rootdir": "${SERVICE_HDFS_DIR}/hbase", + "zookeeper.znode.parent": "${SERVICE_ZK_PATH}", + "hbase.master.hostname": "hbasemaster.${SERVICE_NAME}.${USER}.${DOMAIN}", + "hbase.master.info.port": "16010", + "hbase.regionserver.hostname": "${COMPONENT_INSTANCE_NAME}.${SERVICE_NAME}.${USER}.${DOMAIN}" + } + } + ] + } + } + ], + "quicklinks": { + "HBase Master Status UI": "http://hbasemaster0.${SERVICE_NAME}.${USER}.${DOMAIN}:16010/master-status", + "Proxied HBase Master Status UI": "http://app-proxy/${DOMAIN}/${USER}/${SERVICE_NAME}/hbasemaster/16010/" + } +} +``` diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml new file mode 100644 index 0000000..b9b5b3a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml @@ -0,0 +1,480 @@ +# Hadoop YARN REST APIs for services v1 spec in YAML + +# 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. + +swagger: '2.0' +info: + title: "YARN Simplified API layer for services" + description: | + Bringing a new service on YARN today is not a simple experience. The APIs of + existing frameworks are either too low level (native YARN), require writing + new code (for frameworks with programmatic APIs) or writing a complex spec + (for declarative frameworks). In addition to building critical building blocks + inside YARN (as part of other efforts at YARN-4692), there is a need for + simplifying the user facing story for building services. Experience of projects + like Apache Slider running real-life services like HBase, Storm, Accumulo, + Solr etc, gives us some very good insights on how simplified APIs for services + should look like. + + To this end, we should look at a new simple-services API layer backed by REST + interfaces. This API can be used to create and manage the lifecycle of YARN + services. Services here can range from simple single-component service to + complex multi-component assemblies needing orchestration. YARN-4793 tracks + this effort. + + This document spotlights on this specification. In most of the cases, the + application owner will not be forced to make any changes to their applications. + This is primarily true if the application is packaged with containerization + technologies like docker. Irrespective of how complex the application is, + there will be hooks provided at appropriate layers to allow pluggable and + customizable application behavior. + + version: "1.0.0" + license: + name: Apache 2.0 + url: http://www.apache.org/licenses/LICENSE-2.0.html +# the domain of the service +host: host.mycompany.com +port: 9191(default) +# array of all schemes that your API supports +schemes: + - http +consumes: + - application/json +produces: + - application/json +paths: + /ws/v1/services/version: + get: + summary: Get current version of the API server. + description: Get current version of the API server. + responses: + 200: + description: Successful request + + /ws/v1/services: + get: + summary: (TBD) List of services running in the cluster. + description: Get a list of all currently running services (response includes a minimal projection of the service info). For more details do a GET on a specific service name. + responses: + 200: + description: An array of services + schema: + type: array + items: + $ref: '#/definitions/Service' + default: + description: Unexpected error + schema: + $ref: '#/definitions/ServiceStatus' + post: + summary: Create a service + description: Create a service. The request JSON is a service object with details required for creation. If the request is successful it returns 202 Accepted. A success of this API only confirms success in submission of the service creation request. There is no guarantee that the service will actually reach a RUNNING state. Resource availability and several other factors determines if the service will be deployed in the cluster. It is expected that clients would subsequently call the GET API to get details of the service and determine its state. + parameters: + - name: Service + in: body + description: Service request object + required: true + schema: + $ref: '#/definitions/Service' + responses: + 202: + description: The request to create a service is accepted + 400: + description: Invalid service definition provided in the request body + 500: + description: Failed to create a service + default: + description: Unexpected error + schema: + $ref: '#/definitions/ServiceStatus' + + /ws/v1/services/{service_name}: + put: + summary: Update a service or upgrade the binary version of the components of a running service + description: Update the runtime properties of a service. Currently the following operations are supported - update lifetime, stop/start a service. + The PUT operation is also used to orchestrate an upgrade of the service containers to a newer version of their artifacts (TBD). + parameters: + - name: service_name + in: path + description: Service name + required: true + type: string + - name: Service + in: body + description: The updated service definition. It can contain the updated lifetime of a service or the desired state (STOPPED/STARTED) of a service to initiate a start/stop operation against the specified service + required: true + schema: + $ref: '#/definitions/Service' + responses: + 204: + description: Update or upgrade was successful + 404: + description: Service does not exist + default: + description: Unexpected error + schema: + $ref: '#/definitions/ServiceStatus' + delete: + summary: Destroy a service + description: Destroy a service and release all resources. This API might have to return JSON data providing location of logs (TBD), etc. + parameters: + - name: service_name + in: path + description: Service name + required: true + type: string + responses: + 204: + description: Destroy was successful + 404: + description: Service does not exist + default: + description: Unexpected error + schema: + $ref: '#/definitions/ServiceStatus' + get: + summary: Get details of a service. + description: Return the details (including containers) of a running service + parameters: + - name: service_name + in: path + description: Service name + required: true + type: string + responses: + 200: + description: a service object + schema: + type: object + items: + $ref: '#/definitions/Service' + examples: + service_name: logsearch + artifact: + id: logsearch:latest + type: docker + 404: + description: Service does not exist + default: + description: Unexpected error + schema: + $ref: '#/definitions/ServiceStatus' + /ws/v1/services/{service_name}/components/{component_name}: + put: + summary: Flex a component's number of instances. + description: Set a component's desired number of instanes + parameters: + - name: service_name + in: path + description: Service name + required: true + type: string + - name: component_name + in: path + description: Component name + required: true + type: string + - name: Component + in: body + description: The definition of a component which contains the updated number of instances. + required: true + schema: + $ref: '#/definitions/Component' + responses: + 200: + description: Flex was successful + 404: + description: Service does not exist + default: + description: Unexpected error + schema: + $ref: '#/definitions/ServiceStatus' +definitions: + Service: + description: a service resource has the following attributes. + required: + - name + properties: + name: + type: string + description: A unique service name. If Registry DNS is enabled, the max length is 63 characters. + id: + type: string + description: A unique service id. + artifact: + description: Artifact of single-component service. + $ref: '#/definitions/Artifact' + resource: + description: Resource of single-component service or the global default for multi-component services. Mandatory if it is a single-component service and if cpus and memory are not specified at the Service level. + $ref: '#/definitions/Resource' + launch_command: + type: string + description: The custom launch command of a service component (optional). If not specified for services with docker images say, it will default to the default start command of the image. If there is a single component in this service, you can specify this without the need to have a 'components' section. + launch_time: + type: string + format: date + description: The time when the service was created, e.g. 2016-03-16T01:01:49.000Z. + number_of_containers: + type: integer + format: int64 + description: Number of containers for each component in the service. Each component can further override this service-level global default. + number_of_running_containers: + type: integer + format: int64 + description: In get response this provides the total number of running containers for this service (across all components) at the time of request. Note, a subsequent request can return a different number as and when more containers get allocated until it reaches the total number of containers or if a flex request has been made between the two requests. + lifetime: + type: integer + format: int64 + description: Life time (in seconds) of the service from the time it reaches the STARTED state (after which it is automatically destroyed by YARN). For unlimited lifetime do not set a lifetime value. + placement_policy: + description: (TBD) Advanced scheduling and placement policies. If not specified, it defaults to the default placement policy of the service owner. The design of placement policies are in the works. It is not very clear at this point, how policies in conjunction with labels be exposed to service owners. This is a placeholder for now. The advanced structure of this attribute will be determined by YARN-4902. + $ref: '#/definitions/PlacementPolicy' + components: + description: Components of a service. + type: array + items: + $ref: '#/definitions/Component' + configuration: + description: Config properties of a service. Configurations provided at the service/global level are available to all the components. Specific properties can be overridden at the component level. + $ref: '#/definitions/Configuration' + containers: + description: Containers of a started service. Specifying a value for this attribute for the POST payload raises a validation error. This blob is available only in the GET response of a started service. + type: array + items: + $ref: '#/definitions/Container' + state: + description: State of the service. Specifying a value for this attribute for the POST payload raises a validation error. This attribute is available only in the GET response of a started service. + $ref: '#/definitions/ServiceState' + quicklinks: + type: object + description: A blob of key-value pairs of quicklinks to be exported for a service. + additionalProperties: + type: string + queue: + type: string + description: The YARN queue that this service should be submitted to. + Resource: + description: + Resource determines the amount of resources (vcores, memory, network, etc.) usable by a container. This field determines the resource to be applied for all the containers of a component or service. The resource specified at the service (or global) level can be overriden at the component level. Only one of profile OR cpu & memory are expected. It raises a validation exception otherwise. + properties: + profile: + type: string + description: Each resource profile has a unique id which is associated with a cluster-level predefined memory, cpus, etc. + cpus: + type: integer + format: int32 + description: Amount of vcores allocated to each container (optional but overrides cpus in profile if specified). + memory: + type: string + description: Amount of memory allocated to each container (optional but overrides memory in profile if specified). Currently accepts only an integer value and default unit is in MB. + PlacementPolicy: + description: Placement policy of an instance of a service. This feature is in the works in YARN-6592. + properties: + label: + type: string + description: Assigns a service to a named partition of the cluster where the service desires to run (optional). If not specified all services are submitted to a default label of the service owner. One or more labels can be setup for each service owner account with required constraints like no-preemption, sla-99999, preemption-ok, etc. + Artifact: + description: Artifact of a service component. If not specified, component will just run the bare launch command and no artifact will be localized. + required: + - id + properties: + id: + type: string + description: Artifact id. Examples are package location uri for tarball based services, image name for docker, name of service, etc. + type: + type: string + description: Artifact type, like docker, tarball, etc. (optional). For TARBALL type, the specified tarball will be localized to the container local working directory under a folder named lib. For SERVICE type, the service specified will be read and its components will be added into this service. The original component with artifact type SERVICE will be removed (any properties specified in the original component will be ignored). + enum: + - DOCKER + - TARBALL + - SERVICE + default: DOCKER + uri: + type: string + description: Artifact location to support multiple artifact stores (optional). + Component: + description: One or more components of the service. If the service is HBase say, then the component can be a simple role like master or regionserver. If the service is a complex business webapp then a component can be other services say Kafka or Storm. Thereby it opens up the support for complex and nested services. + required: + - name + properties: + name: + type: string + description: Name of the service component (mandatory). If Registry DNS is enabled, the max length is 63 characters. If unique component support is enabled, the max length is lowered to 44 characters. + dependencies: + type: array + items: + type: string + description: An array of service components which should be in READY state (as defined by readiness check), before this component can be started. The dependencies across all components of a service should be represented as a DAG. + readiness_check: + description: Readiness check for this component. + $ref: '#/definitions/ReadinessCheck' + artifact: + description: Artifact of the component (optional). If not specified, the service level global artifact takes effect. + $ref: '#/definitions/Artifact' + launch_command: + type: string + description: The custom launch command of this component (optional for DOCKER component, required otherwise). When specified at the component level, it overrides the value specified at the global level (if any). + resource: + description: Resource of this component (optional). If not specified, the service level global resource takes effect. + $ref: '#/definitions/Resource' + number_of_containers: + type: integer + format: int64 + description: Number of containers for this component (optional). If not specified, the service level global number_of_containers takes effect. + run_privileged_container: + type: boolean + description: Run all containers of this component in privileged mode (YARN-4262). + placement_policy: + description: Advanced scheduling and placement policies for all containers of this component (optional). If not specified, the service level placement_policy takes effect. Refer to the description at the global level for more details. + $ref: '#/definitions/PlacementPolicy' + configuration: + description: Config properties for this component. + $ref: '#/definitions/Configuration' + quicklinks: + type: array + items: + type: string + description: A list of quicklink keys defined at the service level, and to be resolved by this component. + ReadinessCheck: + description: A custom command or a pluggable helper container to determine the readiness of a container of a component. Readiness for every service is different. Hence the need for a simple interface, with scope to support advanced usecases. + required: + - type + properties: + type: + type: string + description: E.g. HTTP (YARN will perform a simple REST call at a regular interval and expect a 204 No content). + enum: + - HTTP + - PORT + props: + type: object + description: A blob of key value pairs that will be used to configure the check. + additionalProperties: + type: string + artifact: + description: Artifact of the pluggable readiness check helper container (optional). If specified, this helper container typically hosts the http uri and encapsulates the complex scripts required to perform actual container readiness check. At the end it is expected to respond a 204 No content just like the simplified use case. This pluggable framework benefits service owners who can run services without any packaging modifications. Note, artifacts of type docker only is supported for now. NOT IMPLEMENTED YET + $ref: '#/definitions/Artifact' + Configuration: + description: Set of configuration properties that can be injected into the service components via envs, files and custom pluggable helper docker containers. Files of several standard formats like xml, properties, json, yaml and templates will be supported. + properties: + properties: + type: object + description: A blob of key-value pairs of common service properties. + additionalProperties: + type: string + env: + type: object + description: A blob of key-value pairs which will be appended to the default system properties and handed off to the service at start time. All placeholder references to properties will be substituted before injection. + additionalProperties: + type: string + files: + description: Array of list of files that needs to be created and made available as volumes in the service component containers. + type: array + items: + $ref: '#/definitions/ConfigFile' + ConfigFile: + description: A config file that needs to be created and made available as a volume in a service component container. + properties: + type: + type: string + description: Config file in the standard format like xml, properties, json, yaml, template. + enum: + - XML + - PROPERTIES + - JSON + - YAML + - TEMPLATE + - ENV + - HADOOP_XML + dest_file: + type: string + description: The path that this configuration file should be created as. If it is an absolute path, it will be mounted into the DOCKER container. Absolute paths are only allowed for DOCKER containers. If it is a relative path, only the file name should be provided, and the file will be created in the container local working directory under a folder named conf. + src_file: + type: string + description: This provides the source location of the configuration file, the content of which is dumped to dest_file post property substitutions, in the format as specified in type. Typically the src_file would point to a source controlled network accessible file maintained by tools like puppet, chef, or hdfs etc. Currently, only hdfs is supported. + props: + type: object + description: A blob of key value pairs that will be dumped in the dest_file in the format as specified in type. If src_file is specified, src_file content are dumped in the dest_file and these properties will overwrite, if any, existing properties in src_file or be added as new properties in src_file. + Container: + description: An instance of a running service container. + properties: + id: + type: string + description: Unique container id of a running service, e.g. container_e3751_1458061340047_0008_01_000002. + launch_time: + type: string + format: date + description: The time when the container was created, e.g. 2016-03-16T01:01:49.000Z. This will most likely be different from cluster launch time. + ip: + type: string + description: IP address of a running container, e.g. 172.31.42.141. The IP address and hostname attribute values are dependent on the cluster/docker network setup as per YARN-4007. + hostname: + type: string + description: Fully qualified hostname of a running container, e.g. ctr-e3751-1458061340047-0008-01-000002.examplestg.site. The IP address and hostname attribute values are dependent on the cluster/docker network setup as per YARN-4007. + bare_host: + type: string + description: The bare node or host in which the container is running, e.g. cn008.example.com. + state: + description: State of the container of a service. + $ref: '#/definitions/ContainerState' + component_name: + type: string + description: Name of the component that this container instance belongs to. + resource: + description: Resource used for this container. + $ref: '#/definitions/Resource' + artifact: + description: Artifact used for this container. + $ref: '#/definitions/Artifact' + privileged_container: + type: boolean + description: Container running in privileged mode or not. + ServiceState: + description: The current state of a service. + properties: + state: + type: string + description: enum of the state of the service + enum: + - ACCEPTED + - STARTED + - READY + - STOPPED + - FAILED + ContainerState: + description: The current state of the container of a service. + properties: + state: + type: string + description: enum of the state of the container + enum: + - INIT + - STARTED + - READY + ServiceStatus: + description: The current status of a submitted service, returned as a response to the GET API. + properties: + diagnostics: + type: string + description: Diagnostic information (if any) for the reason of the current state of the service. It typically has a non-null value, if the service is in a non-running state. + state: + description: Service state. + $ref: '#/definitions/ServiceState' + code: + type: integer + format: int32 + description: An error code specific to a scenario which service owners should be able to use to understand the failure in addition to the diagnostic information. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/log4j-server.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/log4j-server.properties new file mode 100644 index 0000000..8c679b9 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/log4j-server.properties @@ -0,0 +1,76 @@ +# 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. +# + +# This is the log4j configuration for YARN Services REST API Server + +# Log rotation based on size (100KB) with a max of 10 backup files +log4j.rootLogger=INFO, restservicelog +log4j.threshhold=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 + +log4j.appender.restservicelog=org.apache.log4j.RollingFileAppender +log4j.appender.restservicelog.layout=org.apache.log4j.PatternLayout +log4j.appender.restservicelog.File=${REST_SERVICE_LOG_DIR}/restservice.log +log4j.appender.restservicelog.MaxFileSize=1GB +log4j.appender.restservicelog.MaxBackupIndex=10 + +# log layout skips stack-trace creation operations by avoiding line numbers and method +log4j.appender.restservicelog.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} - %m%n + +# debug edition is much more expensive +#log4j.appender.restservicelog.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n + +# configure stderr +# set the conversion pattern of stderr +# Print the date in ISO 8601 format +log4j.appender.stderr=org.apache.log4j.ConsoleAppender +log4j.appender.stderr.Target=System.err +log4j.appender.stderr.layout=org.apache.log4j.PatternLayout +log4j.appender.stderr.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} - %m%n + +log4j.appender.subprocess=org.apache.log4j.ConsoleAppender +log4j.appender.subprocess.layout=org.apache.log4j.PatternLayout +log4j.appender.subprocess.layout.ConversionPattern=[%c{1}]: %m%n + +# for debugging REST API Service +#log4j.logger.org.apache.hadoop.yarn.services=DEBUG + +# uncomment to debug service lifecycle issues +#log4j.logger.org.apache.hadoop.yarn.service.launcher=DEBUG +#log4j.logger.org.apache.hadoop.yarn.service=DEBUG + +# uncomment for YARN operations +#log4j.logger.org.apache.hadoop.yarn.client=DEBUG + +# uncomment this to debug security problems +#log4j.logger.org.apache.hadoop.security=DEBUG + +#crank back on some noise +log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR +log4j.logger.org.apache.hadoop.hdfs=WARN +log4j.logger.org.apache.hadoop.hdfs.shortcircuit=ERROR + +log4j.logger.org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor=WARN +log4j.logger.org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl=WARN +log4j.logger.org.apache.zookeeper=WARN +log4j.logger.org.apache.curator.framework.state=ERROR +log4j.logger.org.apache.curator.framework.imps=WARN + +log4j.logger.org.mortbay.log=DEBUG diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/webapps/api-server/app b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/webapps/api-server/app new file mode 100644 index 0000000..6a077b1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/webapps/api-server/app @@ -0,0 +1,16 @@ +# 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. + +DON'T DELETE. REST WEBAPP RUN SCRIPT WILL STOP WORKING. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/webapp/WEB-INF/web.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/webapp/WEB-INF/web.xml new file mode 100644 index 0000000..1282c9f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/webapp/WEB-INF/web.xml @@ -0,0 +1,36 @@ + + + + + + Jersey REST API + com.sun.jersey.spi.container.servlet.ServletContainer + + com.sun.jersey.config.property.packages + org.apache.hadoop.yarn.service.webapp,org.apache.hadoop.yarn.service.api,org.apache.hadoop.yarn.service.api.records + + + com.sun.jersey.api.json.POJOMappingFeature + true + + 1 + + + Jersey REST API + /* + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java new file mode 100644 index 0000000..3e08c3a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java @@ -0,0 +1,107 @@ +/* + * 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.service; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.service.api.records.Service; +import org.apache.hadoop.yarn.service.client.ServiceClient; +import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; + +/** + * A mock version of ServiceClient - This class is design + * to simulate various error conditions that will happen + * when a consumer class calls ServiceClient. + */ +public class ServiceClientTest extends ServiceClient { + + private Configuration conf = new Configuration(); + + protected static void init() { + } + + public ServiceClientTest() { + super(); + } + + @Override + public Configuration getConfig() { + return conf; + } + + @Override + public ApplicationId actionCreate(Service service) { + String serviceName = service.getName(); + ServiceApiUtil.validateNameFormat(serviceName, getConfig()); + return ApplicationId.newInstance(System.currentTimeMillis(), 1); + } + + @Override + public Service getStatus(String appName) { + if (appName == null) { + throw new NullPointerException(); + } + if (appName.equals("jenkins")) { + return new Service(); + } else { + throw new IllegalArgumentException(); + } + } + + @Override + public int actionStart(String serviceName) + throws YarnException, IOException { + if (serviceName == null) { + throw new NullPointerException(); + } + if (serviceName.equals("jenkins")) { + return EXIT_SUCCESS; + } else { + throw new ApplicationNotFoundException(""); + } + } + + @Override + public int actionStop(String serviceName, boolean waitForAppStopped) + throws YarnException, IOException { + if (serviceName == null) { + throw new NullPointerException(); + } + if (serviceName.equals("jenkins")) { + return EXIT_SUCCESS; + } else { + throw new ApplicationNotFoundException(""); + } + } + + @Override + public int actionDestroy(String serviceName) { + if (serviceName == null) { + throw new NullPointerException(); + } + if (serviceName.equals("jenkins")) { + return EXIT_SUCCESS; + } else { + throw new IllegalArgumentException(); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/TestApiServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/TestApiServer.java new file mode 100644 index 0000000..2b22474 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/TestApiServer.java @@ -0,0 +1,366 @@ +/* + * 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.service; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.service.api.records.Artifact; +import org.apache.hadoop.yarn.service.api.records.Artifact.TypeEnum; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.api.records.Resource; +import org.apache.hadoop.yarn.service.api.records.Service; +import org.apache.hadoop.yarn.service.api.records.ServiceState; +import org.apache.hadoop.yarn.service.client.ServiceClient; +import org.apache.hadoop.yarn.service.webapp.ApiServer; +import javax.ws.rs.Path; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.Status; + +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.*; + +/** + * Test case for ApiServer REST API. + * + */ +public class TestApiServer { + private ApiServer apiServer; + + @Before + public void setup() throws Exception { + ServiceClient mockServerClient = new ServiceClientTest(); + Configuration conf = new Configuration(); + conf.set("yarn.api-service.service.client.class", + ServiceClientTest.class.getName()); + ApiServer.setServiceClient(mockServerClient); + this.apiServer = new ApiServer(conf); + } + + @Test + public void testPathAnnotation() { + assertNotNull(this.apiServer.getClass().getAnnotation(Path.class)); + assertTrue("The controller has the annotation Path", + this.apiServer.getClass().isAnnotationPresent(Path.class)); + final Path path = this.apiServer.getClass() + .getAnnotation(Path.class); + assertEquals("The path has /ws/v1 annotation", path.value(), + "/ws/v1"); + } + + @Test + public void testGetVersion() { + final Response actual = apiServer.getVersion(); + assertEquals("Version number is", actual.getStatus(), + Response.ok().build().getStatus()); + } + + @Test + public void testBadCreateService() { + Service service = new Service(); + // Test for invalid argument + final Response actual = apiServer.createService(service); + assertEquals("Create service is ", actual.getStatus(), + Response.status(Status.BAD_REQUEST).build().getStatus()); + } + + @Test + public void testGoodCreateService() { + Service service = new Service(); + service.setName("jenkins"); + Artifact artifact = new Artifact(); + artifact.setType(TypeEnum.DOCKER); + artifact.setId("jenkins:latest"); + Resource resource = new Resource(); + resource.setCpus(1); + resource.setMemory("2048"); + List components = new ArrayList(); + Component c = new Component(); + c.setName("jenkins"); + c.setNumberOfContainers(1L); + c.setArtifact(artifact); + c.setLaunchCommand(""); + c.setResource(resource); + components.add(c); + service.setComponents(components); + final Response actual = apiServer.createService(service); + assertEquals("Create service is ", actual.getStatus(), + Response.status(Status.ACCEPTED).build().getStatus()); + } + + @Test + public void testBadGetService() { + final Response actual = apiServer.getService("no-jenkins"); + assertEquals("Get service is ", actual.getStatus(), + Response.status(Status.NOT_FOUND).build().getStatus()); + } + + @Test + public void testBadGetService2() { + final Response actual = apiServer.getService(null); + assertEquals("Get service is ", actual.getStatus(), + Response.status(Status.INTERNAL_SERVER_ERROR) + .build().getStatus()); + } + + @Test + public void testGoodGetService() { + final Response actual = apiServer.getService("jenkins"); + assertEquals("Get service is ", actual.getStatus(), + Response.status(Status.OK).build().getStatus()); + } + + @Test + public void testBadDeleteService() { + final Response actual = apiServer.deleteService("no-jenkins"); + assertEquals("Delete service is ", actual.getStatus(), + Response.status(Status.BAD_REQUEST).build().getStatus()); + } + + @Test + public void testBadDeleteService2() { + final Response actual = apiServer.deleteService(null); + assertEquals("Delete service is ", actual.getStatus(), + Response.status(Status.INTERNAL_SERVER_ERROR) + .build().getStatus()); + } + + @Test + public void testGoodDeleteService() { + final Response actual = apiServer.deleteService("jenkins"); + assertEquals("Delete service is ", actual.getStatus(), + Response.status(Status.OK).build().getStatus()); + } + + @Test + public void testDecreaseContainerAndStop() { + Service service = new Service(); + service.setState(ServiceState.STOPPED); + service.setName("jenkins"); + Artifact artifact = new Artifact(); + artifact.setType(TypeEnum.DOCKER); + artifact.setId("jenkins:latest"); + Resource resource = new Resource(); + resource.setCpus(1); + resource.setMemory("2048"); + List components = new ArrayList(); + Component c = new Component(); + c.setName("jenkins"); + c.setNumberOfContainers(0L); + c.setArtifact(artifact); + c.setLaunchCommand(""); + c.setResource(resource); + components.add(c); + service.setComponents(components); + final Response actual = apiServer.updateService("jenkins", + service); + assertEquals("update service is ", actual.getStatus(), + Response.status(Status.OK).build().getStatus()); + } + + @Test + public void testBadDecreaseContainerAndStop() { + Service service = new Service(); + service.setState(ServiceState.STOPPED); + service.setName("no-jenkins"); + Artifact artifact = new Artifact(); + artifact.setType(TypeEnum.DOCKER); + artifact.setId("jenkins:latest"); + Resource resource = new Resource(); + resource.setCpus(1); + resource.setMemory("2048"); + List components = new ArrayList(); + Component c = new Component(); + c.setName("no-jenkins"); + c.setNumberOfContainers(-1L); + c.setArtifact(artifact); + c.setLaunchCommand(""); + c.setResource(resource); + components.add(c); + service.setComponents(components); + System.out.println("before stop"); + final Response actual = apiServer.updateService("no-jenkins", + service); + assertEquals("flex service is ", actual.getStatus(), + Response.status(Status.BAD_REQUEST).build().getStatus()); + } + + @Test + public void testIncreaseContainersAndStart() { + Service service = new Service(); + service.setState(ServiceState.STARTED); + service.setName("jenkins"); + Artifact artifact = new Artifact(); + artifact.setType(TypeEnum.DOCKER); + artifact.setId("jenkins:latest"); + Resource resource = new Resource(); + resource.setCpus(1); + resource.setMemory("2048"); + List components = new ArrayList(); + Component c = new Component(); + c.setName("jenkins"); + c.setNumberOfContainers(2L); + c.setArtifact(artifact); + c.setLaunchCommand(""); + c.setResource(resource); + components.add(c); + service.setComponents(components); + final Response actual = apiServer.updateService("jenkins", + service); + assertEquals("flex service is ", actual.getStatus(), + Response.status(Status.OK).build().getStatus()); + } + + @Test + public void testBadStartServices() { + Service service = new Service(); + service.setState(ServiceState.STARTED); + service.setName("no-jenkins"); + Artifact artifact = new Artifact(); + artifact.setType(TypeEnum.DOCKER); + artifact.setId("jenkins:latest"); + Resource resource = new Resource(); + resource.setCpus(1); + resource.setMemory("2048"); + List components = new ArrayList(); + Component c = new Component(); + c.setName("jenkins"); + c.setNumberOfContainers(2L); + c.setArtifact(artifact); + c.setLaunchCommand(""); + c.setResource(resource); + components.add(c); + service.setComponents(components); + final Response actual = apiServer.updateService("no-jenkins", + service); + assertEquals("start service is ", actual.getStatus(), + Response.status(Status.INTERNAL_SERVER_ERROR).build() + .getStatus()); + } + + @Test + public void testGoodStartServices() { + Service service = new Service(); + service.setState(ServiceState.STARTED); + service.setName("jenkins"); + Artifact artifact = new Artifact(); + artifact.setType(TypeEnum.DOCKER); + artifact.setId("jenkins:latest"); + Resource resource = new Resource(); + resource.setCpus(1); + resource.setMemory("2048"); + List components = new ArrayList(); + Component c = new Component(); + c.setName("jenkins"); + c.setNumberOfContainers(2L); + c.setArtifact(artifact); + c.setLaunchCommand(""); + c.setResource(resource); + components.add(c); + service.setComponents(components); + final Response actual = apiServer.updateService("jenkins", + service); + assertEquals("start service is ", actual.getStatus(), + Response.status(Status.OK).build().getStatus()); + } + + @Test + public void testBadStopServices() { + Service service = new Service(); + service.setState(ServiceState.STOPPED); + service.setName("no-jenkins"); + Artifact artifact = new Artifact(); + artifact.setType(TypeEnum.DOCKER); + artifact.setId("jenkins:latest"); + Resource resource = new Resource(); + resource.setCpus(1); + resource.setMemory("2048"); + List components = new ArrayList(); + Component c = new Component(); + c.setName("no-jenkins"); + c.setNumberOfContainers(-1L); + c.setArtifact(artifact); + c.setLaunchCommand(""); + c.setResource(resource); + components.add(c); + service.setComponents(components); + System.out.println("before stop"); + final Response actual = apiServer.updateService("no-jenkins", + service); + assertEquals("stop service is ", actual.getStatus(), + Response.status(Status.BAD_REQUEST).build().getStatus()); + } + + @Test + public void testGoodStopServices() { + Service service = new Service(); + service.setState(ServiceState.STARTED); + service.setName("jenkins"); + Artifact artifact = new Artifact(); + artifact.setType(TypeEnum.DOCKER); + artifact.setId("jenkins:latest"); + Resource resource = new Resource(); + resource.setCpus(1); + resource.setMemory("2048"); + List components = new ArrayList(); + Component c = new Component(); + c.setName("jenkins"); + c.setNumberOfContainers(-1L); + c.setArtifact(artifact); + c.setLaunchCommand(""); + c.setResource(resource); + components.add(c); + service.setComponents(components); + System.out.println("before stop"); + final Response actual = apiServer.updateService("jenkins", + service); + assertEquals("stop service is ", actual.getStatus(), + Response.status(Status.OK).build().getStatus()); + } + + @Test + public void testUpdateService() { + Service service = new Service(); + service.setState(ServiceState.STARTED); + service.setName("no-jenkins"); + Artifact artifact = new Artifact(); + artifact.setType(TypeEnum.DOCKER); + artifact.setId("jenkins:latest"); + Resource resource = new Resource(); + resource.setCpus(1); + resource.setMemory("2048"); + List components = new ArrayList(); + Component c = new Component(); + c.setName("no-jenkins"); + c.setNumberOfContainers(-1L); + c.setArtifact(artifact); + c.setLaunchCommand(""); + c.setResource(resource); + components.add(c); + service.setComponents(components); + System.out.println("before stop"); + final Response actual = apiServer.updateService("no-jenkins", + service); + assertEquals("update service is ", actual.getStatus(), + Response.status(Status.INTERNAL_SERVER_ERROR) + .build().getStatus()); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/dev-support/findbugs-exclude.xml new file mode 100644 index 0000000..2814cca --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/dev-support/findbugs-exclude.xml @@ -0,0 +1,48 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/conf/yarnservice-log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/conf/yarnservice-log4j.properties new file mode 100644 index 0000000..58c8e27 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/conf/yarnservice-log4j.properties @@ -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. +# + +# This is the log4j configuration for Slider Application Master + +# Log rotation based on size (256MB) with a max of 20 backup files +log4j.rootLogger=INFO, amlog +log4j.threshhold=ALL +log4j.appender.amlog=org.apache.log4j.RollingFileAppender +log4j.appender.amlog.layout=org.apache.log4j.PatternLayout +log4j.appender.amlog.File=${LOG_DIR}/serviceam.log +log4j.appender.amlog.MaxFileSize=256MB +log4j.appender.amlog.MaxBackupIndex=20 + +# log layout skips stack-trace creation operations by avoiding line numbers and method +log4j.appender.amlog.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} - %m%n + +# debug edition is much more expensive +#log4j.appender.amlog.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n + +# configure stderr +# set the conversion pattern of stderr +# Print the date in ISO 8601 format +log4j.appender.stderr=org.apache.log4j.ConsoleAppender +log4j.appender.stderr.Target=System.err +log4j.appender.stderr.layout=org.apache.log4j.PatternLayout +log4j.appender.stderr.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} - %m%n + +log4j.appender.subprocess=org.apache.log4j.ConsoleAppender +log4j.appender.subprocess.layout=org.apache.log4j.PatternLayout +log4j.appender.subprocess.layout.ConversionPattern=[%c{1}]: %m%n + +# for debugging yarn-service framework +#log4j.logger.org.apache.hadoop.yarn.service=DEBUG + +# uncomment for YARN operations +#log4j.logger.org.apache.hadoop.yarn.client=DEBUG + +# uncomment this to debug security problems +#log4j.logger.org.apache.hadoop.security=DEBUG + +#crank back on some noise +log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR +log4j.logger.org.apache.hadoop.hdfs=WARN + +log4j.logger.org.apache.zookeeper=WARN +log4j.logger.org.apache.curator.framework.state=ERROR +log4j.logger.org.apache.curator.framework.imps=WARN diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd-no-dns/httpd-no-dns.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd-no-dns/httpd-no-dns.json new file mode 100644 index 0000000..6b35538 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd-no-dns/httpd-no-dns.json @@ -0,0 +1,62 @@ +{ + "name": "httpd-service-no-dns", + "lifetime": "3600", + "components": [ + { + "name": "httpd", + "number_of_containers": 2, + "artifact": { + "id": "centos/httpd-24-centos7:latest", + "type": "DOCKER" + }, + "launch_command": "/usr/bin/run-httpd", + "resource": { + "cpus": 1, + "memory": "1024" + }, + "readiness_check": { + "type": "HTTP", + "props": { + "url": "http://${THIS_HOST}:8080" + } + }, + "configuration": { + "files": [ + { + "type": "ENV", + "dest_file": "/var/www/html/index.html", + "props": { + "content": "
Title
Hello from ${COMPONENT_INSTANCE_NAME}!" + } + } + ] + } + }, + { + "name": "httpd-proxy", + "number_of_containers": 1, + "dependencies": [ "httpd" ], + "artifact": { + "id": "centos/httpd-24-centos7:latest", + "type": "DOCKER" + }, + "launch_command": "/usr/bin/run-httpd", + "resource": { + "cpus": 1, + "memory": "1024" + }, + "configuration": { + "files": [ + { + "type": "TEMPLATE", + "dest_file": "/etc/httpd/conf.d/httpd-proxy.conf", + "src_file": "httpd-proxy-no-dns.conf" + } + ] + } + } + ], + "quicklinks": { + "Apache HTTP Server": "http://httpd-proxy-0.${SERVICE_NAME}.${USER}.${DOMAIN}:8080" + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd-no-dns/httpd-proxy-no-dns.conf b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd-no-dns/httpd-proxy-no-dns.conf new file mode 100644 index 0000000..9894e64 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd-no-dns/httpd-proxy-no-dns.conf @@ -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. + + + BalancerMember http://${HTTPD-0_IP}:8080 + BalancerMember http://${HTTPD-1_IP}:8080 + ProxySet lbmethod=bytraffic + + +ProxyPass "/" "balancer://test/" +ProxyPassReverse "/" "balancer://test/" diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd/httpd-proxy.conf b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd/httpd-proxy.conf new file mode 100644 index 0000000..e8651a5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd/httpd-proxy.conf @@ -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. + + + BalancerMember http://httpd-0.${SERVICE_NAME}.${USER}.${DOMAIN}:8080 + BalancerMember http://httpd-1.${SERVICE_NAME}.${USER}.${DOMAIN}:8080 + ProxySet lbmethod=bytraffic + + +ProxyPass "/" "balancer://test/" +ProxyPassReverse "/" "balancer://test/" diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd/httpd.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd/httpd.json new file mode 100644 index 0000000..e63376d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/httpd/httpd.json @@ -0,0 +1,55 @@ +{ + "name": "httpd-service", + "lifetime": "3600", + "components": [ + { + "name": "httpd", + "number_of_containers": 2, + "artifact": { + "id": "centos/httpd-24-centos7:latest", + "type": "DOCKER" + }, + "launch_command": "/usr/bin/run-httpd", + "resource": { + "cpus": 1, + "memory": "1024" + }, + "configuration": { + "files": [ + { + "type": "ENV", + "dest_file": "/var/www/html/index.html", + "props": { + "content": "
Title
Hello from ${COMPONENT_INSTANCE_NAME}!" + } + } + ] + } + }, + { + "name": "httpd-proxy", + "number_of_containers": 1, + "artifact": { + "id": "centos/httpd-24-centos7:latest", + "type": "DOCKER" + }, + "launch_command": "/usr/bin/run-httpd", + "resource": { + "cpus": 1, + "memory": "1024" + }, + "configuration": { + "files": [ + { + "type": "TEMPLATE", + "dest_file": "/etc/httpd/conf.d/httpd-proxy.conf", + "src_file": "httpd-proxy.conf" + } + ] + } + } + ], + "quicklinks": { + "Apache HTTP Server": "http://httpd-proxy-0.${SERVICE_NAME}.${USER}.${DOMAIN}:8080" + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/sleeper/sleeper.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/sleeper/sleeper.json new file mode 100644 index 0000000..89ce527 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/examples/sleeper/sleeper.json @@ -0,0 +1,15 @@ +{ + "name": "sleeper-service", + "components" : + [ + { + "name": "sleeper", + "number_of_containers": 2, + "launch_command": "sleep 900000", + "resource": { + "cpus": 1, + "memory": "256" + } + } + ] +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml new file mode 100644 index 0000000..851f73b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml @@ -0,0 +1,260 @@ + + + 4.0.0 + + org.apache.hadoop + hadoop-yarn-services + 3.1.0-SNAPSHOT + + hadoop-yarn-services-core + jar + Apache Hadoop YARN Services Core + + + + ${project.parent.basedir} + + + + + + + src/main/resources + true + + + + + + org.apache.hadoop + hadoop-maven-plugins + + + compile-protoc + + protoc + + + ${protobuf.version} + ${protoc.path} + + ${basedir}/src/main/proto + + + ${basedir}/src/main/proto + + ClientAMProtocol.proto + + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + ${java.home} + + + + + + org.apache.rat + apache-rat-plugin + + + **/*.json + + + + + + + + + com.beust + jcommander + + + + org.slf4j + slf4j-api + + + + log4j + log4j + runtime + + + + com.google.guava + guava + + + + org.codehaus.jackson + jackson-core-asl + + + + org.codehaus.jackson + jackson-mapper-asl + + + + com.fasterxml.jackson.core + jackson-annotations + + + + org.apache.hadoop + hadoop-hdfs-client + + + + org.apache.hadoop + hadoop-yarn-client + + + + org.apache.hadoop + hadoop-yarn-registry + + + + org.apache.hadoop + hadoop-yarn-common + + + + org.apache.hadoop + hadoop-yarn-server-common + + + + org.apache.hadoop + hadoop-common + + + + org.apache.hadoop + hadoop-annotations + + + + org.apache.hadoop + hadoop-yarn-api + + + + com.google.protobuf + protobuf-java + + + + org.apache.commons + commons-configuration2 + + + + org.apache.commons + commons-compress + + + + commons-io + commons-io + + + + commons-lang + commons-lang + + + + org.apache.curator + curator-client + + + + org.apache.curator + curator-framework + + + + javax.xml.bind + jaxb-api + + + + org.yaml + snakeyaml + + + + io.swagger + swagger-annotations + + + + + + + + junit + junit + test + + + + org.mockito + mockito-all + test + + + + org.apache.hadoop + hadoop-minicluster + test + + + + org.apache.curator + curator-test + test + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMProtocol.java new file mode 100644 index 0000000..516d23d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMProtocol.java @@ -0,0 +1,40 @@ +/** + * 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.service; + +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsResponseProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopResponseProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopRequestProto; + +import java.io.IOException; + +public interface ClientAMProtocol { + FlexComponentsResponseProto flexComponents(FlexComponentsRequestProto request) + throws IOException, YarnException; + + GetStatusResponseProto getStatus(GetStatusRequestProto requestProto) + throws IOException, YarnException; + + StopResponseProto stop(StopRequestProto requestProto) + throws IOException, YarnException; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.java new file mode 100644 index 0000000..8e4c34d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.java @@ -0,0 +1,132 @@ +/** + * 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.service; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.ipc.Server; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.ipc.YarnRPC; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.ComponentCountProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsResponseProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopRequestProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopResponseProto; +import org.apache.hadoop.yarn.service.component.ComponentEvent; +import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetSocketAddress; + +import static org.apache.hadoop.yarn.service.component.ComponentEventType.FLEX; + +public class ClientAMService extends AbstractService + implements ClientAMProtocol { + + private static final Logger LOG = + LoggerFactory.getLogger(ClientAMService.class); + + private ServiceContext context; + private Server server; + + private InetSocketAddress bindAddress; + + public ClientAMService(ServiceContext context) { + super("Client AM Service"); + this.context = context; + } + + @Override protected void serviceStart() throws Exception { + Configuration conf = getConfig(); + YarnRPC rpc = YarnRPC.create(conf); + InetSocketAddress address = new InetSocketAddress(0); + server = rpc.getServer(ClientAMProtocol.class, this, address, conf, + context.secretManager, 1); + server.start(); + + String nodeHostString = + System.getenv(ApplicationConstants.Environment.NM_HOST.name()); + + bindAddress = NetUtils.createSocketAddrForHost(nodeHostString, + server.getListenerAddress().getPort()); + + LOG.info("Instantiated ClientAMService at " + bindAddress); + super.serviceStart(); + } + + @Override protected void serviceStop() throws Exception { + if (server != null) { + server.stop(); + } + super.serviceStop(); + } + + @Override public FlexComponentsResponseProto flexComponents( + FlexComponentsRequestProto request) throws IOException { + if (!request.getComponentsList().isEmpty()) { + for (ComponentCountProto component : request.getComponentsList()) { + ComponentEvent event = new ComponentEvent(component.getName(), FLEX) + .setDesired(component.getNumberOfContainers()); + context.scheduler.getDispatcher().getEventHandler().handle(event); + LOG.info("Flexing component {} to {}", component.getName(), + component.getNumberOfContainers()); + } + } + return FlexComponentsResponseProto.newBuilder().build(); + } + + @Override + public GetStatusResponseProto getStatus(GetStatusRequestProto request) + throws IOException, YarnException { + String stat = ServiceApiUtil.jsonSerDeser.toJson(context.service); + return GetStatusResponseProto.newBuilder().setStatus(stat).build(); + } + + @Override + public StopResponseProto stop(StopRequestProto requestProto) + throws IOException, YarnException { + LOG.info("Stop the service."); + // Stop the service in 2 seconds delay to make sure this rpc call is completed. + // shutdown hook will be executed which will stop AM gracefully. + Thread thread = new Thread() { + @Override + public void run() { + try { + Thread.sleep(2000); + ExitUtil.terminate(0); + } catch (InterruptedException e) { + LOG.error("Interrupted while stopping", e); + } + } + }; + thread.start(); + return StopResponseProto.newBuilder().build(); + } + + public InetSocketAddress getBindAddress() { + return bindAddress; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ContainerFailureTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ContainerFailureTracker.java new file mode 100644 index 0000000..4743f28 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ContainerFailureTracker.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.service; + +import org.apache.hadoop.yarn.service.component.Component; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.NODE_BLACKLIST_THRESHOLD; + +/** + * This tracks the container failures per node. If the failure counter exceeds + * the maxFailurePerNode limit, it'll blacklist that node. + * + */ +public class ContainerFailureTracker { + + private static final Logger LOG = + LoggerFactory.getLogger(ContainerFailureTracker.class); + + // Host -> num container failures + private Map failureCountPerNode = new HashMap<>(); + private Set blackListedNodes = new HashSet<>(); + private ServiceContext context; + private int maxFailurePerNode; + private Component component; + + public ContainerFailureTracker(ServiceContext context, Component component) { + this.context = context; + this.component = component; + maxFailurePerNode = component.getComponentSpec().getConfiguration() + .getPropertyInt(NODE_BLACKLIST_THRESHOLD, 3); + } + + + public synchronized void incNodeFailure(String host) { + int num = 0; + if (failureCountPerNode.containsKey(host)) { + num = failureCountPerNode.get(host); + } + num++; + failureCountPerNode.put(host, num); + + // black list the node if exceed max failure + if (num > maxFailurePerNode && !blackListedNodes.contains(host)) { + List blacklists = new ArrayList<>(); + blacklists.add(host); + blackListedNodes.add(host); + context.scheduler.getAmRMClient().updateBlacklist(blacklists, null); + LOG.info("[COMPONENT {}]: Failed {} times on this host, blacklisted {}." + + " Current list of blacklisted nodes: {}", + component.getName(), num, host, blackListedNodes); + } + } + + public synchronized void resetContainerFailures() { + // reset container failure counter per node + failureCountPerNode.clear(); + context.scheduler.getAmRMClient() + .updateBlacklist(null, new ArrayList<>(blackListedNodes)); + LOG.info("[COMPONENT {}]: Clearing blacklisted nodes {} ", + component.getName(), blackListedNodes); + blackListedNodes.clear(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceContext.java new file mode 100644 index 0000000..94dbc6e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceContext.java @@ -0,0 +1,41 @@ +/** + * 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.service; + +import com.google.common.cache.LoadingCache; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager; +import org.apache.hadoop.yarn.service.api.records.Service; +import org.apache.hadoop.yarn.service.api.records.ConfigFile; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; + +public class ServiceContext { + public Service service = null; + public SliderFileSystem fs; + public String serviceHdfsDir = ""; + public ApplicationAttemptId attemptId; + public LoadingCache configCache; + public ServiceScheduler scheduler; + public ClientToAMTokenSecretManager secretManager; + public ClientAMService clientAMService; + + public ServiceContext() { + + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java new file mode 100644 index 0000000..2abdae1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java @@ -0,0 +1,156 @@ +/** + * 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.service; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.util.GenericOptionsParser; +import org.apache.hadoop.util.ShutdownHookManager; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler; +import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager; +import org.apache.hadoop.yarn.service.client.params.ServiceAMArgs; +import org.apache.hadoop.yarn.service.monitor.ServiceMonitor; +import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; +import org.apache.hadoop.yarn.service.utils.SliderUtils; +import org.apache.hadoop.yarn.service.exceptions.BadClusterStateException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; + +public class ServiceMaster extends CompositeService { + + private static final Logger LOG = + LoggerFactory.getLogger(ServiceMaster.class); + + private static ServiceAMArgs amArgs; + protected ServiceContext context; + + public ServiceMaster(String name) { + super(name); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + //TODO Deprecate slider conf, make sure works with yarn conf + printSystemEnv(); + if (UserGroupInformation.isSecurityEnabled()) { + UserGroupInformation.setConfiguration(conf); + } + LOG.info("Login user is {}", UserGroupInformation.getLoginUser()); + + context = new ServiceContext(); + Path appDir = getAppDir(); + context.serviceHdfsDir = appDir.toString(); + SliderFileSystem fs = new SliderFileSystem(conf); + context.fs = fs; + fs.setAppDir(appDir); + loadApplicationJson(context, fs); + + ContainerId amContainerId = getAMContainerId(); + + ApplicationAttemptId attemptId = amContainerId.getApplicationAttemptId(); + LOG.info("Service AppAttemptId: " + attemptId); + context.attemptId = attemptId; + + // configure AM to wait forever for RM + conf.setLong(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS, -1); + conf.unset(YarnConfiguration.CLIENT_FAILOVER_MAX_ATTEMPTS); + + DefaultMetricsSystem.initialize("ServiceAppMaster"); + + context.secretManager = new ClientToAMTokenSecretManager(attemptId, null); + ClientAMService clientAMService = new ClientAMService(context); + context.clientAMService = clientAMService; + addService(clientAMService); + + ServiceScheduler scheduler = createServiceScheduler(context); + addService(scheduler); + context.scheduler = scheduler; + + ServiceMonitor monitor = new ServiceMonitor("Service Monitor", context); + addService(monitor); + + super.serviceInit(conf); + } + + protected ContainerId getAMContainerId() throws BadClusterStateException { + return ContainerId.fromString(SliderUtils.mandatoryEnvVariable( + ApplicationConstants.Environment.CONTAINER_ID.name())); + } + + protected Path getAppDir() { + return new Path(amArgs.getServiceDefPath()).getParent(); + } + + protected ServiceScheduler createServiceScheduler(ServiceContext context) + throws IOException, YarnException { + return new ServiceScheduler(context); + } + + protected void loadApplicationJson(ServiceContext context, + SliderFileSystem fs) throws IOException { + context.service = ServiceApiUtil + .loadServiceFrom(fs, new Path(amArgs.getServiceDefPath())); + LOG.info(context.service.toString()); + } + + @Override + protected void serviceStop() throws Exception { + LOG.info("Stopping app master"); + super.serviceStop(); + } + + private void printSystemEnv() { + for (Map.Entry envs : System.getenv().entrySet()) { + LOG.info("{} = {}", envs.getKey(), envs.getValue()); + } + } + + public static void main(String[] args) throws Exception { + Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); + StringUtils.startupShutdownMessage(ServiceMaster.class, args, LOG); + amArgs = new ServiceAMArgs(args); + amArgs.parse(); + try { + ServiceMaster serviceMaster = new ServiceMaster("Service Master"); + ShutdownHookManager.get() + .addShutdownHook(new CompositeServiceShutdownHook(serviceMaster), 30); + YarnConfiguration conf = new YarnConfiguration(); + new GenericOptionsParser(conf, args); + serviceMaster.init(conf); + serviceMaster.start(); + } catch (Throwable t) { + LOG.error("Error starting service master", t); + ExitUtil.terminate(1, "Error starting service master"); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMetrics.java new file mode 100644 index 0000000..9fc886e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMetrics.java @@ -0,0 +1,98 @@ +/* + * 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.service; + +import org.apache.hadoop.metrics2.MetricsCollector; +import org.apache.hadoop.metrics2.MetricsInfo; +import org.apache.hadoop.metrics2.MetricsSource; +import org.apache.hadoop.metrics2.annotation.Metric; +import org.apache.hadoop.metrics2.annotation.Metrics; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.metrics2.lib.MetricsRegistry; +import org.apache.hadoop.metrics2.lib.MutableGaugeInt; + +import static org.apache.hadoop.metrics2.lib.Interns.info; + +@Metrics(context = "yarn-native-service") +public class ServiceMetrics implements MetricsSource { + + @Metric("containers requested") + public MutableGaugeInt containersRequested; + + @Metric("anti-affinity containers pending") + public MutableGaugeInt pendingAAContainers; + + @Metric("containers running") + public MutableGaugeInt containersRunning; + + @Metric("containers ready") + public MutableGaugeInt containersReady; + + @Metric("containers desired") + public MutableGaugeInt containersDesired; + + @Metric("containers succeeded") + public MutableGaugeInt containersSucceeded; + + @Metric("containers failed") + public MutableGaugeInt containersFailed; + + @Metric("containers preempted") + public MutableGaugeInt containersPreempted; + + @Metric("containers surplus") + public MutableGaugeInt surplusContainers; + + @Metric("containers failed due to disk failure") + public MutableGaugeInt containersDiskFailure; + + protected final MetricsRegistry registry; + + public ServiceMetrics(MetricsInfo metricsInfo) { + registry = new MetricsRegistry(metricsInfo); + } + + @Override + public void getMetrics(MetricsCollector collector, boolean all) { + registry.snapshot(collector.addRecord(registry.info()), all); + } + + public static ServiceMetrics register(String name, String description) { + ServiceMetrics metrics = new ServiceMetrics(info(name, description)); + DefaultMetricsSystem.instance().register(name, description, metrics); + return metrics; + } + + public void tag(String name, String description, String value) { + registry.tag(name, description, value); + } + + @Override public String toString() { + return "ServiceMetrics{" + + "containersRequested=" + containersRequested.value() + + ", pendingAAContainers=" + pendingAAContainers.value() + + ", containersRunning=" + containersRunning.value() + + ", containersDesired=" + containersDesired.value() + + ", containersSucceeded=" + containersSucceeded.value() + + ", containersFailed=" + containersFailed.value() + + ", containersPreempted=" + containersPreempted.value() + + ", surplusContainers=" + surplusContainers.value() + '}'; + } +} + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java new file mode 100644 index 0000000..f3824df --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java @@ -0,0 +1,705 @@ +/** + * 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.service; + +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.registry.client.api.RegistryOperations; +import org.apache.hadoop.registry.client.api.RegistryOperationsFactory; +import org.apache.hadoop.registry.client.binding.RegistryPathUtils; +import org.apache.hadoop.registry.client.binding.RegistryUtils; +import org.apache.hadoop.registry.client.types.ServiceRecord; +import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies; +import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.UpdatedContainer; +import org.apache.hadoop.yarn.client.api.AMRMClient; +import org.apache.hadoop.yarn.client.api.TimelineV2Client; +import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; +import org.apache.hadoop.yarn.client.api.async.NMClientAsync; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.service.api.ServiceApiConstants; +import org.apache.hadoop.yarn.service.api.records.Service; +import org.apache.hadoop.yarn.service.api.records.ConfigFile; +import org.apache.hadoop.yarn.service.component.instance.ComponentInstance; +import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent; +import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType; +import org.apache.hadoop.yarn.service.component.Component; +import org.apache.hadoop.yarn.service.component.ComponentEvent; +import org.apache.hadoop.yarn.service.component.ComponentEventType; +import org.apache.hadoop.yarn.service.conf.YarnServiceConstants; +import org.apache.hadoop.yarn.service.containerlaunch.ContainerLaunchService; +import org.apache.hadoop.yarn.service.provider.ProviderUtils; +import org.apache.hadoop.yarn.service.registry.YarnRegistryViewForProviders; +import org.apache.hadoop.yarn.service.timelineservice.ServiceMetricsSink; +import org.apache.hadoop.yarn.service.timelineservice.ServiceTimelinePublisher; +import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; +import org.apache.hadoop.yarn.service.utils.ServiceRegistryUtils; +import org.apache.hadoop.yarn.util.BoundedAppender; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.URI; +import java.nio.ByteBuffer; +import java.text.MessageFormat; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY; +import static org.apache.hadoop.registry.client.api.RegistryConstants.*; +import static org.apache.hadoop.yarn.service.api.ServiceApiConstants.*; +import static org.apache.hadoop.yarn.service.component.ComponentEventType.*; + +/** + * + */ +public class ServiceScheduler extends CompositeService { + + private static final Logger LOG = + LoggerFactory.getLogger(ServiceScheduler.class); + private Service app; + + // component_name -> component + private final Map componentsByName = + new ConcurrentHashMap<>(); + + // id - > component + protected final Map componentsById = + new ConcurrentHashMap<>(); + + private final Map liveInstances = + new ConcurrentHashMap<>(); + + private ServiceMetrics serviceMetrics; + + private ServiceTimelinePublisher serviceTimelinePublisher; + + // Global diagnostics that will be reported to RM on eRxit. + // The unit the number of characters. This will be limited to 64 * 1024 + // characters. + private BoundedAppender diagnostics = new BoundedAppender(64 * 1024); + + // A cache for loading config files from remote such as hdfs + public LoadingCache configFileCache = null; + + public ScheduledExecutorService executorService; + public Map globalTokens = new HashMap<>(); + + private AMRMClientAsync amRMClient; + private NMClientAsync nmClient; + private AsyncDispatcher dispatcher; + AsyncDispatcher compInstanceDispatcher; + private YarnRegistryViewForProviders yarnRegistryOperations; + private ServiceContext context; + private ContainerLaunchService containerLaunchService; + + public ServiceScheduler(ServiceContext context) { + super(context.service.getName()); + this.context = context; + } + + public void buildInstance(ServiceContext context, Configuration configuration) + throws YarnException { + app = context.service; + executorService = Executors.newScheduledThreadPool(10); + RegistryOperations registryClient = RegistryOperationsFactory + .createInstance("ServiceScheduler", configuration); + addIfService(registryClient); + yarnRegistryOperations = + createYarnRegistryOperations(context, registryClient); + + // register metrics + serviceMetrics = ServiceMetrics + .register(app.getName(), "Metrics for service"); + serviceMetrics.tag("type", "Metrics type [component or service]", "service"); + serviceMetrics.tag("appId", "Service id for service", app.getId()); + + amRMClient = createAMRMClient(); + addIfService(amRMClient); + + nmClient = createNMClient(); + addIfService(nmClient); + + dispatcher = new AsyncDispatcher("Component dispatcher"); + dispatcher.register(ComponentEventType.class, + new ComponentEventHandler()); + dispatcher.setDrainEventsOnStop(); + addIfService(dispatcher); + + compInstanceDispatcher = + new AsyncDispatcher("CompInstance dispatcher"); + compInstanceDispatcher.register(ComponentInstanceEventType.class, + new ComponentInstanceEventHandler()); + addIfService(compInstanceDispatcher); + containerLaunchService = new ContainerLaunchService(context.fs); + addService(containerLaunchService); + + if (YarnConfiguration.timelineServiceV2Enabled(configuration)) { + TimelineV2Client timelineClient = TimelineV2Client + .createTimelineClient(context.attemptId.getApplicationId()); + amRMClient.registerTimelineV2Client(timelineClient); + serviceTimelinePublisher = new ServiceTimelinePublisher(timelineClient); + addService(serviceTimelinePublisher); + DefaultMetricsSystem.instance().register("ServiceMetricsSink", + "For processing metrics to ATS", + new ServiceMetricsSink(serviceTimelinePublisher)); + LOG.info("Timeline v2 is enabled."); + } + + initGlobalTokensForSubstitute(context); + //substitute quicklinks + ProviderUtils.substituteMapWithTokens(app.getQuicklinks(), globalTokens); + createConfigFileCache(context.fs.getFileSystem()); + + createAllComponents(); + } + + protected YarnRegistryViewForProviders createYarnRegistryOperations( + ServiceContext context, RegistryOperations registryClient) { + return new YarnRegistryViewForProviders(registryClient, + RegistryUtils.currentUser(), YarnServiceConstants.APP_TYPE, app.getName(), + context.attemptId); + } + + protected NMClientAsync createNMClient() { + return NMClientAsync.createNMClientAsync(new NMClientCallback()); + } + + protected AMRMClientAsync createAMRMClient() { + return AMRMClientAsync + .createAMRMClientAsync(1000, new AMRMClientCallback()); + } + + @Override + public void serviceInit(Configuration conf) throws Exception { + try { + buildInstance(context, conf); + } catch (YarnException e) { + throw new YarnRuntimeException(e); + } + super.serviceInit(conf); + } + + @Override + public void serviceStop() throws Exception { + LOG.info("Stopping service scheduler"); + + if (executorService != null) { + executorService.shutdownNow(); + } + + DefaultMetricsSystem.shutdown(); + if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) { + serviceTimelinePublisher + .serviceAttemptUnregistered(context, diagnostics.toString()); + } + String msg = diagnostics.toString() + + "Navigate to the failed component for more details."; + amRMClient + .unregisterApplicationMaster(FinalApplicationStatus.ENDED, msg, ""); + LOG.info("Service " + app.getName() + + " unregistered with RM, with attemptId = " + context.attemptId + + ", diagnostics = " + diagnostics); + super.serviceStop(); + } + + @Override + public void serviceStart() throws Exception { + super.serviceStart(); + InetSocketAddress bindAddress = context.clientAMService.getBindAddress(); + RegisterApplicationMasterResponse response = amRMClient + .registerApplicationMaster(bindAddress.getHostName(), + bindAddress.getPort(), "N/A"); + if (response.getClientToAMTokenMasterKey() != null + && response.getClientToAMTokenMasterKey().remaining() != 0) { + context.secretManager + .setMasterKey(response.getClientToAMTokenMasterKey().array()); + } + registerServiceInstance(context.attemptId, app); + + // recover components based on containers sent from RM + recoverComponents(response); + + for (Component component : componentsById.values()) { + // Trigger initial evaluation of components + if (component.areDependenciesReady()) { + LOG.info("Triggering initial evaluation of component {}", + component.getName()); + ComponentEvent event = new ComponentEvent(component.getName(), FLEX) + .setDesired(component.getComponentSpec().getNumberOfContainers()); + component.handle(event); + } + } + } + + private void recoverComponents(RegisterApplicationMasterResponse response) { + List recoveredContainers = response + .getContainersFromPreviousAttempts(); + LOG.info("Received {} containers from previous attempt.", + recoveredContainers.size()); + Map existingRecords = new HashMap<>(); + List existingComps = null; + try { + existingComps = yarnRegistryOperations.listComponents(); + LOG.info("Found {} containers from ZK registry: {}", existingComps.size(), + existingComps); + } catch (Exception e) { + LOG.info("Could not read component paths: {}", e.getMessage()); + } + if (existingComps != null) { + for (String existingComp : existingComps) { + try { + ServiceRecord record = + yarnRegistryOperations.getComponent(existingComp); + existingRecords.put(existingComp, record); + } catch (Exception e) { + LOG.warn("Could not resolve record for component {}: {}", + existingComp, e); + } + } + } + for (Container container : recoveredContainers) { + LOG.info("Handling container {} from previous attempt", + container.getId()); + ServiceRecord record = existingRecords.get(RegistryPathUtils + .encodeYarnID(container.getId().toString())); + if (record != null) { + Component comp = componentsById.get(container.getAllocationRequestId()); + ComponentEvent event = + new ComponentEvent(comp.getName(), CONTAINER_RECOVERED) + .setContainer(container) + .setInstance(comp.getComponentInstance(record.description)); + comp.handle(event); + // do not remove requests in this case because we do not know if they + // have already been removed + } else { + LOG.info("Record not found in registry for container {} from previous" + + " attempt, releasing", container.getId()); + amRMClient.releaseAssignedContainer(container.getId()); + } + } + } + + private void initGlobalTokensForSubstitute(ServiceContext context) { + // ZK + globalTokens.put(ServiceApiConstants.CLUSTER_ZK_QUORUM, getConfig() + .getTrimmed(KEY_REGISTRY_ZK_QUORUM, DEFAULT_REGISTRY_ZK_QUORUM)); + String user = null; + try { + user = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch (IOException e) { + LOG.error("Failed to get user.", e); + } + globalTokens + .put(SERVICE_ZK_PATH, ServiceRegistryUtils.mkClusterPath(user, app.getName())); + + globalTokens.put(ServiceApiConstants.USER, user); + String dnsDomain = getConfig().getTrimmed(KEY_DNS_DOMAIN); + if (dnsDomain != null && !dnsDomain.isEmpty()) { + globalTokens.put(ServiceApiConstants.DOMAIN, dnsDomain); + } + // HDFS + String clusterFs = getConfig().getTrimmed(FS_DEFAULT_NAME_KEY); + if (clusterFs != null && !clusterFs.isEmpty()) { + globalTokens.put(ServiceApiConstants.CLUSTER_FS_URI, clusterFs); + globalTokens.put(ServiceApiConstants.CLUSTER_FS_HOST, + URI.create(clusterFs).getHost()); + } + globalTokens.put(SERVICE_HDFS_DIR, context.serviceHdfsDir); + // service name + globalTokens.put(SERVICE_NAME_LC, app.getName().toLowerCase()); + globalTokens.put(SERVICE_NAME, app.getName()); + } + + private void createConfigFileCache(final FileSystem fileSystem) { + this.configFileCache = + CacheBuilder.newBuilder().expireAfterAccess(10, TimeUnit.MINUTES) + .build(new CacheLoader() { + @Override public Object load(ConfigFile key) throws Exception { + switch (key.getType()) { + case HADOOP_XML: + try (FSDataInputStream input = fileSystem + .open(new Path(key.getSrcFile()))) { + org.apache.hadoop.conf.Configuration confRead = + new org.apache.hadoop.conf.Configuration(false); + confRead.addResource(input); + Map map = new HashMap<>(confRead.size()); + for (Map.Entry entry : confRead) { + map.put(entry.getKey(), entry.getValue()); + } + return map; + } + case TEMPLATE: + try (FSDataInputStream fileInput = fileSystem + .open(new Path(key.getSrcFile()))) { + return IOUtils.toString(fileInput); + } + default: + return null; + } + } + }); + context.configCache = configFileCache; + } + + private void registerServiceInstance(ApplicationAttemptId attemptId, + Service service) throws IOException { + LOG.info("Registering " + attemptId + ", " + service.getName() + + " into registry"); + ServiceRecord serviceRecord = new ServiceRecord(); + serviceRecord.set(YarnRegistryAttributes.YARN_ID, + attemptId.getApplicationId().toString()); + serviceRecord.set(YarnRegistryAttributes.YARN_PERSISTENCE, + PersistencePolicies.APPLICATION); + serviceRecord.description = "YarnServiceMaster"; + + // set any provided attributes + setUserProvidedServiceRecordAttributes(service.getConfiguration(), + serviceRecord); + + executorService.submit(new Runnable() { + @Override public void run() { + try { + yarnRegistryOperations.registerSelf(serviceRecord, false); + LOG.info("Registered service under {}; absolute path {}", + yarnRegistryOperations.getSelfRegistrationPath(), + yarnRegistryOperations.getAbsoluteSelfRegistrationPath()); + boolean isFirstAttempt = 1 == attemptId.getAttemptId(); + // delete the children in case there are any and this is an AM startup. + // just to make sure everything underneath is purged + if (isFirstAttempt) { + yarnRegistryOperations.deleteChildren( + yarnRegistryOperations.getSelfRegistrationPath(), true); + } + } catch (IOException e) { + LOG.error( + "Failed to register app " + app.getName() + " in registry"); + } + } + }); + if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) { + serviceTimelinePublisher.serviceAttemptRegistered(app, getConfig()); + } + } + + private void setUserProvidedServiceRecordAttributes( + org.apache.hadoop.yarn.service.api.records.Configuration conf, ServiceRecord record) { + String prefix = "service.record.attribute"; + for (Map.Entry entry : conf.getProperties().entrySet()) { + if (entry.getKey().startsWith(prefix)) { + String key = entry.getKey().substring(prefix.length() + 1); + record.set(key, entry.getValue().trim()); + } + } + } + + private void createAllComponents() { + long allocateId = 0; + + // sort components by dependencies + Collection sortedComponents = + ServiceApiUtil.sortByDependencies(app.getComponents()); + + for (org.apache.hadoop.yarn.service.api.records.Component compSpec : sortedComponents) { + Component component = new Component(compSpec, allocateId, context); + componentsById.put(allocateId, component); + componentsByName.put(component.getName(), component); + allocateId++; + } + } + + private final class ComponentEventHandler + implements EventHandler { + @Override + public void handle(ComponentEvent event) { + Component component = componentsByName.get(event.getName()); + + if (component == null) { + LOG.error("No component exists for " + event.getName()); + return; + } + try { + component.handle(event); + } catch (Throwable t) { + LOG.error(MessageFormat + .format("[COMPONENT {0}]: Error in handling event type {1}", + component.getName(), event.getType()), t); + } + } + } + + private final class ComponentInstanceEventHandler + implements EventHandler { + @Override + public void handle(ComponentInstanceEvent event) { + ComponentInstance instance = + liveInstances.get(event.getContainerId()); + if (instance == null) { + LOG.error("No component instance exists for " + event.getContainerId()); + return; + } + try { + instance.handle(event); + } catch (Throwable t) { + LOG.error(instance.getCompInstanceId() + + ": Error in handling event type " + event.getType(), t); + } + } + } + + class AMRMClientCallback extends AMRMClientAsync.AbstractCallbackHandler { + + @Override + public void onContainersAllocated(List containers) { + LOG.info(containers.size() + " containers allocated. "); + for (Container container : containers) { + Component comp = componentsById.get(container.getAllocationRequestId()); + ComponentEvent event = + new ComponentEvent(comp.getName(), CONTAINER_ALLOCATED) + .setContainer(container); + dispatcher.getEventHandler().handle(event); + Collection requests = amRMClient + .getMatchingRequests(container.getAllocationRequestId()); + LOG.info("[COMPONENT {}]: {} outstanding container requests.", + comp.getName(), requests.size()); + // remove the corresponding request + if (requests.iterator().hasNext()) { + LOG.info("[COMPONENT {}]: removing one container request.", comp + .getName()); + AMRMClient.ContainerRequest request = requests.iterator().next(); + amRMClient.removeContainerRequest(request); + } + } + } + + @Override + public void onContainersCompleted(List statuses) { + for (ContainerStatus status : statuses) { + ContainerId containerId = status.getContainerId(); + ComponentInstance instance = liveInstances.get(status.getContainerId()); + if (instance == null) { + LOG.warn( + "Container {} Completed. No component instance exists. exitStatus={}. diagnostics={} ", + containerId, status.getExitStatus(), status.getDiagnostics()); + return; + } + ComponentEvent event = + new ComponentEvent(instance.getCompName(), CONTAINER_COMPLETED) + .setStatus(status).setInstance(instance); + dispatcher.getEventHandler().handle(event); + } + } + + @Override + public void onContainersUpdated(List containers) { + } + + @Override public void onShutdownRequest() { + //Was used for non-work-preserving restart in YARN, should be deprecated. + } + + @Override public void onNodesUpdated(List updatedNodes) { + StringBuilder str = new StringBuilder(); + str.append("Nodes updated info: ").append(System.lineSeparator()); + for (NodeReport report : updatedNodes) { + str.append(report.getNodeId()).append(", state = ") + .append(report.getNodeState()).append(", healthDiagnostics = ") + .append(report.getHealthReport()).append(System.lineSeparator()); + } + LOG.warn(str.toString()); + } + + @Override public float getProgress() { + // get running containers over desired containers + long total = 0; + for (org.apache.hadoop.yarn.service.api.records.Component component : app + .getComponents()) { + total += component.getNumberOfContainers(); + } + // Probably due to user flexed down to 0 + if (total == 0) { + return 100; + } + return Math.max((float) liveInstances.size() / total * 100, 100); + } + + @Override public void onError(Throwable e) { + LOG.error("Error in AMRMClient callback handler ", e); + } + } + + + private class NMClientCallback extends NMClientAsync.AbstractCallbackHandler { + + @Override public void onContainerStarted(ContainerId containerId, + Map allServiceResponse) { + ComponentInstance instance = liveInstances.get(containerId); + if (instance == null) { + LOG.error("No component instance exists for " + containerId); + return; + } + ComponentEvent event = + new ComponentEvent(instance.getCompName(), CONTAINER_STARTED) + .setInstance(instance); + dispatcher.getEventHandler().handle(event); + } + + @Override public void onContainerStatusReceived(ContainerId containerId, + ContainerStatus containerStatus) { + + } + + @Override public void onContainerStopped(ContainerId containerId) { + + } + + @Override + public void onStartContainerError(ContainerId containerId, Throwable t) { + ComponentInstance instance = liveInstances.get(containerId); + if (instance == null) { + LOG.error("No component instance exists for " + containerId); + return; + } + amRMClient.releaseAssignedContainer(containerId); + // After container released, it'll get CONTAINER_COMPLETED event from RM + // automatically which will trigger stopping COMPONENT INSTANCE + } + + @Override public void onContainerResourceIncreased(ContainerId containerId, + Resource resource) { + + } + + @Override public void onContainerResourceUpdated(ContainerId containerId, + Resource resource) { + + } + + @Override public void onGetContainerStatusError(ContainerId containerId, + Throwable t) { + + } + + @Override + public void onIncreaseContainerResourceError(ContainerId containerId, + Throwable t) { + + } + + @Override + public void onUpdateContainerResourceError(ContainerId containerId, + Throwable t) { + + } + + @Override + public void onStopContainerError(ContainerId containerId, Throwable t) { + + } + } + + public ServiceMetrics getServiceMetrics() { + return serviceMetrics; + } + + public AMRMClientAsync getAmRMClient() { + return amRMClient; + } + + public NMClientAsync getNmClient() { + return nmClient; + } + + public void addLiveCompInstance(ContainerId containerId, + ComponentInstance instance) { + liveInstances.put(containerId, instance); + } + + public void removeLiveCompInstance(ContainerId containerId) { + liveInstances.remove(containerId); + } + + public AsyncDispatcher getCompInstanceDispatcher() { + return compInstanceDispatcher; + } + + public YarnRegistryViewForProviders getYarnRegistryOperations() { + return yarnRegistryOperations; + } + + public ServiceTimelinePublisher getServiceTimelinePublisher() { + return serviceTimelinePublisher; + } + + public Map getLiveInstances() { + return liveInstances; + } + + public ContainerLaunchService getContainerLaunchService() { + return containerLaunchService; + } + + public ServiceContext getContext() { + return context; + } + + public Map getAllComponents() { + return componentsByName; + } + + public Service getApp() { + return app; + } + + public AsyncDispatcher getDispatcher() { + return dispatcher; + } + + public BoundedAppender getDiagnostics() { + return diagnostics; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/ServiceApiConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/ServiceApiConstants.java new file mode 100644 index 0000000..a85191c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/ServiceApiConstants.java @@ -0,0 +1,74 @@ +/* + * 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.service.api; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import static org.apache.hadoop.yarn.service.utils.ServiceApiUtil.$; + +/** + * This class defines constants that can be used in input spec for + * variable substitutions + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public interface ServiceApiConstants { + + // Constants for service + String SERVICE_NAME = $("SERVICE_NAME"); + + String SERVICE_NAME_LC = $("SERVICE_NAME.lc"); + + String USER = $("USER"); + + String DOMAIN = $("DOMAIN"); + + // Constants for component + String COMPONENT_NAME = $("COMPONENT_NAME"); + + String COMPONENT_NAME_LC = $("COMPONENT_NAME.lc"); + + String COMPONENT_INSTANCE_NAME = $("COMPONENT_INSTANCE_NAME"); + + // Constants for component instance + String COMPONENT_ID = $("COMPONENT_ID"); + + String CONTAINER_ID = $("CONTAINER_ID"); + + // Templates for component instance host/IP + String COMPONENT_HOST = $("%s_HOST"); + + String COMPONENT_IP = $("%s_IP"); + + // Constants for default cluster ZK + String CLUSTER_ZK_QUORUM = $("CLUSTER_ZK_QUORUM"); + + // URI for the default cluster fs + String CLUSTER_FS_URI = $("CLUSTER_FS_URI"); + + // the host component of the cluster fs UI + String CLUSTER_FS_HOST = $("CLUSTER_FS_HOST"); + + // Path in zookeeper for a specific service + String SERVICE_ZK_PATH = $("SERVICE_ZK_PATH"); + + // Constants for service specific hdfs dir + String SERVICE_HDFS_DIR = $("SERVICE_HDFS_DIR"); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Artifact.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Artifact.java new file mode 100644 index 0000000..ce062cc --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Artifact.java @@ -0,0 +1,168 @@ +/* + * 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.service.api.records; + +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; + +import java.io.Serializable; +import java.util.Objects; + +import javax.xml.bind.annotation.XmlEnum; +import javax.xml.bind.annotation.XmlType; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonValue; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Artifact of an service component. + **/ +@InterfaceAudience.Public +@InterfaceStability.Unstable +@ApiModel(description = "Artifact of an service component") +@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") +@JsonInclude(JsonInclude.Include.NON_NULL) +public class Artifact implements Serializable { + private static final long serialVersionUID = 3608929500111099035L; + + private String id = null; + + /** + * Artifact Type. DOCKER, TARBALL or SERVICE + **/ + @XmlType(name = "artifact_type") + @XmlEnum + public enum TypeEnum { + DOCKER("DOCKER"), TARBALL("TARBALL"), SERVICE("SERVICE"); + + private String value; + + TypeEnum(String value) { + this.value = value; + } + + @Override + @JsonValue + public String toString() { + return value; + } + } + + private TypeEnum type = TypeEnum.DOCKER; + private String uri = null; + + /** + * Artifact id. Examples are package location uri for tarball based services, + * image name for docker, etc. + **/ + public Artifact id(String id) { + this.id = id; + return this; + } + + @ApiModelProperty(example = "null", required = true, value = "Artifact id. Examples are package location uri for tarball based services, image name for docker, etc.") + @JsonProperty("id") + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * Artifact type, like docker, tarball, etc. (optional). + **/ + public Artifact type(TypeEnum type) { + this.type = type; + return this; + } + + @ApiModelProperty(example = "null", value = "Artifact type, like docker, tarball, etc. (optional).") + @JsonProperty("type") + public TypeEnum getType() { + return type; + } + + public void setType(TypeEnum type) { + this.type = type; + } + + /** + * Artifact location to support multiple artifact stores (optional). + **/ + public Artifact uri(String uri) { + this.uri = uri; + return this; + } + + @ApiModelProperty(example = "null", value = "Artifact location to support multiple artifact stores (optional).") + @JsonProperty("uri") + public String getUri() { + return uri; + } + + public void setUri(String uri) { + this.uri = uri; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Artifact artifact = (Artifact) o; + return Objects.equals(this.id, artifact.id) + && Objects.equals(this.type, artifact.type) + && Objects.equals(this.uri, artifact.uri); + } + + @Override + public int hashCode() { + return Objects.hash(id, type, uri); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("class Artifact {\n"); + + sb.append(" id: ").append(toIndentedString(id)).append("\n"); + sb.append(" type: ").append(toIndentedString(type)).append("\n"); + sb.append(" uri: ").append(toIndentedString(uri)).append("\n"); + sb.append("}"); + return sb.toString(); + } + + /** + * Convert the given object to string with each line indented by 4 spaces + * (except the first line). + */ + private String toIndentedString(java.lang.Object o) { + if (o == null) { + return "null"; + } + return o.toString().replace("\n", "\n "); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/BaseResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/BaseResource.java new file mode 100644 index 0000000..7ac86d4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/BaseResource.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.service.api.records; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import java.io.Serializable; +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class BaseResource implements Serializable { + private static final long serialVersionUID = 1492603053176889431L; + + private String uri; + + /** + * Resource location for a service, e.g. + * /ws/v1/services/helloworld + * + **/ + public String getUri() { + return uri; + } + + public void setUri(String uri) { + this.uri = uri; + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("BaseResource [uri="); + builder.append(uri); + builder.append("]"); + return builder.toString(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Component.java new file mode 100644 index 0000000..b0a8e82 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Component.java @@ -0,0 +1,412 @@ +/* + * 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.service.api.records; + +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +import javax.xml.bind.annotation.XmlElement; +import javax.xml.bind.annotation.XmlRootElement; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * One or more components of the service. If the service is HBase say, + * then the component can be a simple role like master or regionserver. If the + * service is a complex business webapp then a component can be other + * services say Kafka or Storm. Thereby it opens up the support for complex + * and nested services. + **/ +@InterfaceAudience.Public +@InterfaceStability.Unstable +@ApiModel(description = "One or more components of the service. If the service is HBase say, then the component can be a simple role like master or regionserver. If the service is a complex business webapp then a component can be other services say Kafka or Storm. Thereby it opens up the support for complex and nested services.") +@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") +@XmlRootElement +@JsonInclude(JsonInclude.Include.NON_NULL) +public class Component implements Serializable { + private static final long serialVersionUID = -8430058381509087805L; + + private String name = null; + private List dependencies = new ArrayList(); + private ReadinessCheck readinessCheck = null; + private Artifact artifact = null; + private String launchCommand = null; + private Resource resource = null; + private Long numberOfContainers = null; + private Boolean runPrivilegedContainer = false; + private PlacementPolicy placementPolicy = null; + private Configuration configuration = new Configuration(); + private List quicklinks = new ArrayList(); + private List containers = + Collections.synchronizedList(new ArrayList()); + + /** + * Name of the service component (mandatory). + **/ + public Component name(String name) { + this.name = name; + return this; + } + + @ApiModelProperty(example = "null", required = true, value = "Name of the service component (mandatory).") + @JsonProperty("name") + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * An array of service components which should be in READY state (as + * defined by readiness check), before this component can be started. The + * dependencies across all components of a service should be represented + * as a DAG. + **/ + public Component dependencies(List dependencies) { + this.dependencies = dependencies; + return this; + } + + @ApiModelProperty(example = "null", value = "An array of service components which should be in READY state (as defined by readiness check), before this component can be started. The dependencies across all components of an service should be represented as a DAG.") + @JsonProperty("dependencies") + public List getDependencies() { + return dependencies; + } + + public void setDependencies(List dependencies) { + this.dependencies = dependencies; + } + + /** + * Readiness check for this component. + **/ + public Component readinessCheck(ReadinessCheck readinessCheck) { + this.readinessCheck = readinessCheck; + return this; + } + + @ApiModelProperty(example = "null", value = "Readiness check for this component.") + @JsonProperty("readiness_check") + public ReadinessCheck getReadinessCheck() { + return readinessCheck; + } + + @XmlElement(name = "readiness_check") + public void setReadinessCheck(ReadinessCheck readinessCheck) { + this.readinessCheck = readinessCheck; + } + + /** + * Artifact of the component (optional). If not specified, the service + * level global artifact takes effect. + **/ + public Component artifact(Artifact artifact) { + this.artifact = artifact; + return this; + } + + @ApiModelProperty(example = "null", value = "Artifact of the component (optional). If not specified, the service level global artifact takes effect.") + @JsonProperty("artifact") + public Artifact getArtifact() { + return artifact; + } + + public void setArtifact(Artifact artifact) { + this.artifact = artifact; + } + + /** + * The custom launch command of this component (optional). When specified at + * the component level, it overrides the value specified at the global level + * (if any). + **/ + public Component launchCommand(String launchCommand) { + this.launchCommand = launchCommand; + return this; + } + + @ApiModelProperty(example = "null", value = "The custom launch command of this component (optional). When specified at the component level, it overrides the value specified at the global level (if any).") + @JsonProperty("launch_command") + public String getLaunchCommand() { + return launchCommand; + } + + @XmlElement(name = "launch_command") + public void setLaunchCommand(String launchCommand) { + this.launchCommand = launchCommand; + } + + /** + * Resource of this component (optional). If not specified, the service + * level global resource takes effect. + **/ + public Component resource(Resource resource) { + this.resource = resource; + return this; + } + + @ApiModelProperty(example = "null", value = "Resource of this component (optional). If not specified, the service level global resource takes effect.") + @JsonProperty("resource") + public Resource getResource() { + return resource; + } + + public void setResource(Resource resource) { + this.resource = resource; + } + + /** + * Number of containers for this component (optional). If not specified, + * the service level global number_of_containers takes effect. + **/ + public Component numberOfContainers(Long numberOfContainers) { + this.numberOfContainers = numberOfContainers; + return this; + } + + @ApiModelProperty(example = "null", value = "Number of containers for this component (optional). If not specified, the service level global number_of_containers takes effect.") + @JsonProperty("number_of_containers") + public Long getNumberOfContainers() { + return numberOfContainers; + } + + @XmlElement(name = "number_of_containers") + public void setNumberOfContainers(Long numberOfContainers) { + this.numberOfContainers = numberOfContainers; + } + + @ApiModelProperty(example = "null", value = "Containers of a started component. Specifying a value for this attribute for the POST payload raises a validation error. This blob is available only in the GET response of a started service.") + @JsonProperty("containers") + public List getContainers() { + return containers; + } + + public void setContainers(List containers) { + this.containers = containers; + } + + public void addContainer(Container container) { + this.containers.add(container); + } + + public void removeContainer(Container container) { + containers.remove(container); + } + public Container getContainer(String id) { + for (Container container : containers) { + if (container.getId().equals(id)) { + return container; + } + } + return null; + } + + /** + * Run all containers of this component in privileged mode (YARN-4262). + **/ + public Component runPrivilegedContainer(Boolean runPrivilegedContainer) { + this.runPrivilegedContainer = runPrivilegedContainer; + return this; + } + + @ApiModelProperty(example = "null", value = "Run all containers of this component in privileged mode (YARN-4262).") + @JsonProperty("run_privileged_container") + public Boolean getRunPrivilegedContainer() { + return runPrivilegedContainer; + } + + @XmlElement(name = "run_privileged_container") + public void setRunPrivilegedContainer(Boolean runPrivilegedContainer) { + this.runPrivilegedContainer = runPrivilegedContainer; + } + + /** + * Advanced scheduling and placement policies for all containers of this + * component (optional). If not specified, the service level placement_policy + * takes effect. Refer to the description at the global level for more + * details. + **/ + public Component placementPolicy(PlacementPolicy placementPolicy) { + this.placementPolicy = placementPolicy; + return this; + } + + @ApiModelProperty(example = "null", value = "Advanced scheduling and placement policies for all containers of this component (optional). If not specified, the service level placement_policy takes effect. Refer to the description at the global level for more details.") + @JsonProperty("placement_policy") + public PlacementPolicy getPlacementPolicy() { + return placementPolicy; + } + + @XmlElement(name = "placement_policy") + public void setPlacementPolicy(PlacementPolicy placementPolicy) { + this.placementPolicy = placementPolicy; + } + + /** + * Config properties for this component. + **/ + public Component configuration(Configuration configuration) { + this.configuration = configuration; + return this; + } + + @ApiModelProperty(example = "null", value = "Config properties for this component.") + @JsonProperty("configuration") + public Configuration getConfiguration() { + return configuration; + } + + public void setConfiguration(Configuration configuration) { + this.configuration = configuration; + } + + /** + * A list of quicklink keys defined at the service level, and to be + * resolved by this component. + **/ + public Component quicklinks(List quicklinks) { + this.quicklinks = quicklinks; + return this; + } + + @ApiModelProperty(example = "null", value = "A list of quicklink keys defined at the service level, and to be resolved by this component.") + @JsonProperty("quicklinks") + public List getQuicklinks() { + return quicklinks; + } + + public void setQuicklinks(List quicklinks) { + this.quicklinks = quicklinks; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Component component = (Component) o; + return Objects.equals(this.name, component.name) + && Objects.equals(this.dependencies, component.dependencies) + && Objects.equals(this.readinessCheck, component.readinessCheck) + && Objects.equals(this.artifact, component.artifact) + && Objects.equals(this.launchCommand, component.launchCommand) + && Objects.equals(this.resource, component.resource) + && Objects.equals(this.numberOfContainers, component.numberOfContainers) + && Objects.equals(this.runPrivilegedContainer, + component.runPrivilegedContainer) + && Objects.equals(this.placementPolicy, component.placementPolicy) + && Objects.equals(this.configuration, component.configuration) + && Objects.equals(this.quicklinks, component.quicklinks); + } + + @Override + public int hashCode() { + return Objects.hash(name, dependencies, readinessCheck, artifact, + launchCommand, resource, numberOfContainers, + runPrivilegedContainer, placementPolicy, configuration, quicklinks); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("class Component {\n"); + + sb.append(" name: ").append(toIndentedString(name)).append("\n"); + sb.append(" dependencies: ").append(toIndentedString(dependencies)) + .append("\n"); + sb.append(" readinessCheck: ").append(toIndentedString(readinessCheck)) + .append("\n"); + sb.append(" artifact: ").append(toIndentedString(artifact)).append("\n"); + sb.append(" launchCommand: ").append(toIndentedString(launchCommand)) + .append("\n"); + sb.append(" resource: ").append(toIndentedString(resource)).append("\n"); + sb.append(" numberOfContainers: ") + .append(toIndentedString(numberOfContainers)).append("\n"); + sb.append(" containers: ").append(toIndentedString(containers)) + .append("\n"); + sb.append(" runPrivilegedContainer: ") + .append(toIndentedString(runPrivilegedContainer)).append("\n"); + sb.append(" placementPolicy: ").append(toIndentedString(placementPolicy)) + .append("\n"); + sb.append(" configuration: ").append(toIndentedString(configuration)) + .append("\n"); + sb.append(" quicklinks: ").append(toIndentedString(quicklinks)) + .append("\n"); + sb.append("}"); + return sb.toString(); + } + + /** + * Convert the given object to string with each line indented by 4 spaces + * (except the first line). + */ + private String toIndentedString(java.lang.Object o) { + if (o == null) { + return "null"; + } + return o.toString().replace("\n", "\n "); + } + + /** + * Merge from another component into this component without overwriting. + */ + public void mergeFrom(Component that) { + if (this.getArtifact() == null) { + this.setArtifact(that.getArtifact()); + } + if (this.getResource() == null) { + this.setResource(that.getResource()); + } + if (this.getNumberOfContainers() == null) { + this.setNumberOfContainers(that.getNumberOfContainers()); + } + if (this.getLaunchCommand() == null) { + this.setLaunchCommand(that.getLaunchCommand()); + } + this.getConfiguration().mergeFrom(that.getConfiguration()); + if (this.getQuicklinks() == null) { + this.setQuicklinks(that.getQuicklinks()); + } + if (this.getRunPrivilegedContainer() == null) { + this.setRunPrivilegedContainer(that.getRunPrivilegedContainer()); + } + if (this.getDependencies() == null) { + this.setDependencies(that.getDependencies()); + } + if (this.getPlacementPolicy() == null) { + this.setPlacementPolicy(that.getPlacementPolicy()); + } + if (this.getReadinessCheck() == null) { + this.setReadinessCheck(that.getReadinessCheck()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFile.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFile.java new file mode 100644 index 0000000..fb088e2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFile.java @@ -0,0 +1,233 @@ +/* + * 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.service.api.records; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonValue; +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import javax.xml.bind.annotation.XmlElement; +import javax.xml.bind.annotation.XmlEnum; +import javax.xml.bind.annotation.XmlRootElement; +import javax.xml.bind.annotation.XmlType; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +/** + * A config file that needs to be created and made available as a volume in an + * service component container. + **/ +@InterfaceAudience.Public +@InterfaceStability.Unstable +@ApiModel(description = "A config file that needs to be created and made available as a volume in an service component container.") +@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") +@XmlRootElement +@JsonInclude(JsonInclude.Include.NON_NULL) +public class ConfigFile implements Serializable { + private static final long serialVersionUID = -7009402089417704612L; + + /** + * Config Type. XML, JSON, YAML, TEMPLATE, ENV and HADOOP_XML are supported. + **/ + @XmlType(name = "config_type") + @XmlEnum + public enum TypeEnum { + XML("XML"), PROPERTIES("PROPERTIES"), JSON("JSON"), YAML("YAML"), TEMPLATE( + "TEMPLATE"), ENV("ENV"), HADOOP_XML("HADOOP_XML"),; + + private String value; + + TypeEnum(String value) { + this.value = value; + } + + @Override + @JsonValue + public String toString() { + return value; + } + } + + private TypeEnum type = null; + private String destFile = null; + private String srcFile = null; + private Map props = new HashMap<>(); + + public ConfigFile copy() { + ConfigFile copy = new ConfigFile(); + copy.setType(this.getType()); + copy.setSrcFile(this.getSrcFile()); + copy.setDestFile(this.getDestFile()); + if (this.getProps() != null && !this.getProps().isEmpty()) { + copy.getProps().putAll(this.getProps()); + } + return copy; + } + + /** + * Config file in the standard format like xml, properties, json, yaml, + * template. + **/ + public ConfigFile type(TypeEnum type) { + this.type = type; + return this; + } + + @ApiModelProperty(example = "null", value = "Config file in the standard format like xml, properties, json, yaml, template.") + @JsonProperty("type") + public TypeEnum getType() { + return type; + } + + public void setType(TypeEnum type) { + this.type = type; + } + + /** + * The absolute path that this configuration file should be mounted as, in the + * service container. + **/ + public ConfigFile destFile(String destFile) { + this.destFile = destFile; + return this; + } + + @ApiModelProperty(example = "null", value = "The absolute path that this configuration file should be mounted as, in the service container.") + @JsonProperty("dest_file") + public String getDestFile() { + return destFile; + } + + @XmlElement(name = "dest_file") + public void setDestFile(String destFile) { + this.destFile = destFile; + } + + /** + * This provides the source location of the configuration file, the content + * of which is dumped to dest_file post property substitutions, in the format + * as specified in type. Typically the src_file would point to a source + * controlled network accessible file maintained by tools like puppet, chef, + * or hdfs etc. Currently, only hdfs is supported. + **/ + public ConfigFile srcFile(String srcFile) { + this.srcFile = srcFile; + return this; + } + + @ApiModelProperty(example = "null", value = "This provides the source location of the configuration file, " + + "the content of which is dumped to dest_file post property substitutions, in the format as specified in type. " + + "Typically the src_file would point to a source controlled network accessible file maintained by tools like puppet, chef, or hdfs etc. Currently, only hdfs is supported.") + @JsonProperty("src_file") + public String getSrcFile() { + return srcFile; + } + + @XmlElement(name = "src_file") + public void setSrcFile(String srcFile) { + this.srcFile = srcFile; + } + + /** + A blob of key value pairs that will be dumped in the dest_file in the format + as specified in type. If src_file is specified, src_file content are dumped + in the dest_file and these properties will overwrite, if any, existing + properties in src_file or be added as new properties in src_file. + **/ + public ConfigFile props(Map props) { + this.props = props; + return this; + } + + @ApiModelProperty(example = "null", value = "A blob of key value pairs that will be dumped in the dest_file in the format as specified in type." + + " If src_file is specified, src_file content are dumped in the dest_file and these properties will overwrite, if any," + + " existing properties in src_file or be added as new properties in src_file.") + @JsonProperty("props") + public Map getProps() { + return props; + } + + public void setProps(Map props) { + this.props = props; + } + + public long getLong(String name, long defaultValue) { + if (name == null) { + return defaultValue; + } + String value = props.get(name.trim()); + return Long.parseLong(value); + } + + public boolean getBoolean(String name, boolean defaultValue) { + if (name == null) { + return defaultValue; + } + return Boolean.valueOf(props.get(name.trim())); + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ConfigFile configFile = (ConfigFile) o; + return Objects.equals(this.type, configFile.type) + && Objects.equals(this.destFile, configFile.destFile) + && Objects.equals(this.srcFile, configFile.srcFile); + } + + @Override + public int hashCode() { + return Objects.hash(type, destFile, srcFile, props); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("class ConfigFile {\n"); + + sb.append(" type: ").append(toIndentedString(type)).append("\n"); + sb.append(" destFile: ").append(toIndentedString(destFile)).append("\n"); + sb.append(" srcFile: ").append(toIndentedString(srcFile)).append("\n"); + sb.append(" props: ").append(toIndentedString(props)).append("\n"); + sb.append("}"); + return sb.toString(); + } + + /** + * Convert the given object to string with each line indented by 4 spaces + * (except the first line). + */ + private String toIndentedString(java.lang.Object o) { + if (o == null) { + return "null"; + } + return o.toString().replace("\n", "\n "); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFormat.java new file mode 100644 index 0000000..e10305a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFormat.java @@ -0,0 +1,67 @@ +/* + * 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.service.api.records; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import java.util.Locale; + +@InterfaceAudience.Public +@InterfaceStability.Unstable +public enum ConfigFormat { + + JSON("json"), + PROPERTIES("properties"), + XML("xml"), + HADOOP_XML("hadoop_xml"), + ENV("env"), + TEMPLATE("template"), + YAML("yaml"), + ; + ConfigFormat(String suffix) { + this.suffix = suffix; + } + + private final String suffix; + + public String getSuffix() { + return suffix; + } + + + @Override + public String toString() { + return suffix; + } + + /** + * Get a matching format or null + * @param type + * @return the format + */ + public static ConfigFormat resolve(String type) { + for (ConfigFormat format: values()) { + if (format.getSuffix().equals(type.toLowerCase(Locale.ENGLISH))) { + return format; + } + } + return null; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Configuration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Configuration.java new file mode 100644 index 0000000..2f8ca96 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Configuration.java @@ -0,0 +1,225 @@ +/* + * 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.service.api.records; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.service.utils.SliderUtils; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Set of configuration properties that can be injected into the service + * components via envs, files and custom pluggable helper docker containers. + * Files of several standard formats like xml, properties, json, yaml and + * templates will be supported. + **/ +@InterfaceAudience.Public +@InterfaceStability.Unstable +@ApiModel(description = "Set of configuration properties that can be injected into the service components via envs, files and custom pluggable helper docker containers. Files of several standard formats like xml, properties, json, yaml and templates will be supported.") +@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") +@JsonInclude(JsonInclude.Include.NON_NULL) +public class Configuration implements Serializable { + private static final long serialVersionUID = -4330788704981074466L; + + private Map properties = new HashMap(); + private Map env = new HashMap(); + private List files = new ArrayList(); + + /** + * A blob of key-value pairs of common service properties. + **/ + public Configuration properties(Map properties) { + this.properties = properties; + return this; + } + + @ApiModelProperty(example = "null", value = "A blob of key-value pairs of common service properties.") + @JsonProperty("properties") + public Map getProperties() { + return properties; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + /** + * A blob of key-value pairs which will be appended to the default system + * properties and handed off to the service at start time. All placeholder + * references to properties will be substituted before injection. + **/ + public Configuration env(Map env) { + this.env = env; + return this; + } + + @ApiModelProperty(example = "null", value = "A blob of key-value pairs which will be appended to the default system properties and handed off to the service at start time. All placeholder references to properties will be substituted before injection.") + @JsonProperty("env") + public Map getEnv() { + return env; + } + + public void setEnv(Map env) { + this.env = env; + } + + /** + * Array of list of files that needs to be created and made available as + * volumes in the service component containers. + **/ + public Configuration files(List files) { + this.files = files; + return this; + } + + @ApiModelProperty(example = "null", value = "Array of list of files that needs to be created and made available as volumes in the service component containers.") + @JsonProperty("files") + public List getFiles() { + return files; + } + + public void setFiles(List files) { + this.files = files; + } + + public long getPropertyLong(String name, long defaultValue) { + String value = getProperty(name); + if (StringUtils.isEmpty(value)) { + return defaultValue; + } + return Long.parseLong(value); + } + + public int getPropertyInt(String name, int defaultValue) { + String value = getProperty(name); + if (StringUtils.isEmpty(value)) { + return defaultValue; + } + return Integer.parseInt(value); + } + + public boolean getPropertyBool(String name, boolean defaultValue) { + String value = getProperty(name); + if (StringUtils.isEmpty(value)) { + return defaultValue; + } + return Boolean.parseBoolean(value); + } + + public String getProperty(String name, String defaultValue) { + String value = getProperty(name); + if (StringUtils.isEmpty(value)) { + return defaultValue; + } + return value; + } + + public void setProperty(String name, String value) { + properties.put(name, value); + } + + public String getProperty(String name) { + return properties.get(name.trim()); + } + + public String getEnv(String name) { + return env.get(name.trim()); + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Configuration configuration = (Configuration) o; + return Objects.equals(this.properties, configuration.properties) + && Objects.equals(this.env, configuration.env) + && Objects.equals(this.files, configuration.files); + } + + @Override + public int hashCode() { + return Objects.hash(properties, env, files); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("class Configuration {\n"); + + sb.append(" properties: ").append(toIndentedString(properties)) + .append("\n"); + sb.append(" env: ").append(toIndentedString(env)).append("\n"); + sb.append(" files: ").append(toIndentedString(files)).append("\n"); + sb.append("}"); + return sb.toString(); + } + + /** + * Convert the given object to string with each line indented by 4 spaces + * (except the first line). + */ + private String toIndentedString(java.lang.Object o) { + if (o == null) { + return "null"; + } + return o.toString().replace("\n", "\n "); + } + + /** + * Merge all properties and envs from that configuration to this configration. + * For ConfigFiles, all properties and envs of that ConfigFile are merged into + * this ConfigFile. + */ + public synchronized void mergeFrom(Configuration that) { + SliderUtils.mergeMapsIgnoreDuplicateKeys(this.properties, that + .getProperties()); + SliderUtils.mergeMapsIgnoreDuplicateKeys(this.env, that.getEnv()); + + Map thatMap = new HashMap<>(); + for (ConfigFile file : that.getFiles()) { + thatMap.put(file.getDestFile(), file.copy()); + } + for (ConfigFile thisFile : files) { + if(thatMap.containsKey(thisFile.getDestFile())) { + ConfigFile thatFile = thatMap.get(thisFile.getDestFile()); + SliderUtils.mergeMapsIgnoreDuplicateKeys(thisFile.getProps(), + thatFile.getProps()); + thatMap.remove(thisFile.getDestFile()); + } + } + // add remaining new files from that Configration + for (ConfigFile thatFile : thatMap.values()) { + files.add(thatFile.copy()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Container.java new file mode 100644 index 0000000..cf8cd79 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Container.java @@ -0,0 +1,297 @@ +/* + * 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.service.api.records; + +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; + +import java.util.Date; +import java.util.Objects; + +import javax.xml.bind.annotation.XmlElement; +import javax.xml.bind.annotation.XmlRootElement; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * An instance of a running service container. + **/ +@InterfaceAudience.Public +@InterfaceStability.Unstable +@ApiModel(description = "An instance of a running service container") +@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") +@XmlRootElement +@JsonInclude(JsonInclude.Include.NON_NULL) +public class Container extends BaseResource { + private static final long serialVersionUID = -8955788064529288L; + + private String id = null; + private Date launchTime = null; + private String ip = null; + private String hostname = null; + private String bareHost = null; + private ContainerState state = null; + private String componentName = null; + private Resource resource = null; + private Artifact artifact = null; + private Boolean privilegedContainer = null; + + /** + * Unique container id of a running service, e.g. + * container_e3751_1458061340047_0008_01_000002. + **/ + public Container id(String id) { + this.id = id; + return this; + } + + @ApiModelProperty(example = "null", value = "Unique container id of a running service, e.g. container_e3751_1458061340047_0008_01_000002.") + @JsonProperty("id") + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The time when the container was created, e.g. 2016-03-16T01:01:49.000Z. + * This will most likely be different from cluster launch time. + **/ + public Container launchTime(Date launchTime) { + this.launchTime = launchTime == null ? null : (Date) launchTime.clone(); + return this; + } + + @ApiModelProperty(example = "null", value = "The time when the container was created, e.g. 2016-03-16T01:01:49.000Z. This will most likely be different from cluster launch time.") + @JsonProperty("launch_time") + public Date getLaunchTime() { + return launchTime == null ? null : (Date) launchTime.clone(); + } + + @XmlElement(name = "launch_time") + public void setLaunchTime(Date launchTime) { + this.launchTime = launchTime == null ? null : (Date) launchTime.clone(); + } + + /** + * IP address of a running container, e.g. 172.31.42.141. The IP address and + * hostname attribute values are dependent on the cluster/docker network setup + * as per YARN-4007. + **/ + public Container ip(String ip) { + this.ip = ip; + return this; + } + + @ApiModelProperty(example = "null", value = "IP address of a running container, e.g. 172.31.42.141. The IP address and hostname attribute values are dependent on the cluster/docker network setup as per YARN-4007.") + @JsonProperty("ip") + public String getIp() { + return ip; + } + + public void setIp(String ip) { + this.ip = ip; + } + + /** + * Fully qualified hostname of a running container, e.g. + * ctr-e3751-1458061340047-0008-01-000002.examplestg.site. The IP address and + * hostname attribute values are dependent on the cluster/docker network setup + * as per YARN-4007. + **/ + public Container hostname(String hostname) { + this.hostname = hostname; + return this; + } + + @ApiModelProperty(example = "null", value = "Fully qualified hostname of a running container, e.g. ctr-e3751-1458061340047-0008-01-000002.examplestg.site. The IP address and hostname attribute values are dependent on the cluster/docker network setup as per YARN-4007.") + @JsonProperty("hostname") + public String getHostname() { + return hostname; + } + + public void setHostname(String hostname) { + this.hostname = hostname; + } + + /** + * The bare node or host in which the container is running, e.g. + * cn008.example.com. + **/ + public Container bareHost(String bareHost) { + this.bareHost = bareHost; + return this; + } + + @ApiModelProperty(example = "null", value = "The bare node or host in which the container is running, e.g. cn008.example.com.") + @JsonProperty("bare_host") + public String getBareHost() { + return bareHost; + } + + @XmlElement(name = "bare_host") + public void setBareHost(String bareHost) { + this.bareHost = bareHost; + } + + /** + * State of the container of an service. + **/ + public Container state(ContainerState state) { + this.state = state; + return this; + } + + @ApiModelProperty(example = "null", value = "State of the container of an service.") + @JsonProperty("state") + public ContainerState getState() { + return state; + } + + public void setState(ContainerState state) { + this.state = state; + } + + /** + * Name of the component that this container instance belongs to. + **/ + public Container componentName(String componentName) { + this.componentName = componentName; + return this; + } + + @ApiModelProperty(example = "null", value = "Name of the component that this container instance belongs to.") + @JsonProperty("component_name") + public String getComponentName() { + return componentName; + } + + @XmlElement(name = "component_name") + public void setComponentName(String componentName) { + this.componentName = componentName; + } + + /** + * Resource used for this container. + **/ + public Container resource(Resource resource) { + this.resource = resource; + return this; + } + + @ApiModelProperty(example = "null", value = "Resource used for this container.") + @JsonProperty("resource") + public Resource getResource() { + return resource; + } + + public void setResource(Resource resource) { + this.resource = resource; + } + + /** + * Artifact used for this container. + **/ + public Container artifact(Artifact artifact) { + this.artifact = artifact; + return this; + } + + @ApiModelProperty(example = "null", value = "Artifact used for this container.") + @JsonProperty("artifact") + public Artifact getArtifact() { + return artifact; + } + + public void setArtifact(Artifact artifact) { + this.artifact = artifact; + } + + /** + * Container running in privileged mode or not. + **/ + public Container privilegedContainer(Boolean privilegedContainer) { + this.privilegedContainer = privilegedContainer; + return this; + } + + @ApiModelProperty(example = "null", value = "Container running in privileged mode or not.") + @JsonProperty("privileged_container") + public Boolean getPrivilegedContainer() { + return privilegedContainer; + } + + public void setPrivilegedContainer(Boolean privilegedContainer) { + this.privilegedContainer = privilegedContainer; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Container container = (Container) o; + return Objects.equals(this.id, container.id); + } + + @Override + public int hashCode() { + return Objects.hash(id); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("class Container {\n"); + + sb.append(" id: ").append(toIndentedString(id)).append("\n"); + sb.append(" launchTime: ").append(toIndentedString(launchTime)) + .append("\n"); + sb.append(" ip: ").append(toIndentedString(ip)).append("\n"); + sb.append(" hostname: ").append(toIndentedString(hostname)).append("\n"); + sb.append(" bareHost: ").append(toIndentedString(bareHost)).append("\n"); + sb.append(" state: ").append(toIndentedString(state)).append("\n"); + sb.append(" componentName: ").append(toIndentedString(componentName)) + .append("\n"); + sb.append(" resource: ").append(toIndentedString(resource)).append("\n"); + sb.append(" artifact: ").append(toIndentedString(artifact)).append("\n"); + sb.append(" privilegedContainer: ") + .append(toIndentedString(privilegedContainer)).append("\n"); + sb.append("}"); + return sb.toString(); + } + + /** + * Convert the given object to string with each line indented by 4 spaces + * (except the first line). + */ + private String toIndentedString(java.lang.Object o) { + if (o == null) { + return "null"; + } + return o.toString().replace("\n", "\n "); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ContainerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ContainerState.java new file mode 100644 index 0000000..bf09ff2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ContainerState.java @@ -0,0 +1,30 @@ +/* + * 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.service.api.records; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * The current state of the container of an application. + **/ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public enum ContainerState { + RUNNING_BUT_UNREADY, READY, STOPPED +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Error.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Error.java new file mode 100644 index 0000000..c64b1b5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Error.java @@ -0,0 +1,129 @@ +/* + * 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.service.api.records; + +import io.swagger.annotations.ApiModelProperty; + +import java.util.Objects; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +@InterfaceAudience.Public +@InterfaceStability.Unstable +@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") +public class Error { + + private Integer code = null; + private String message = null; + private String fields = null; + + /** + **/ + public Error code(Integer code) { + this.code = code; + return this; + } + + @ApiModelProperty(example = "null", value = "") + @JsonProperty("code") + public Integer getCode() { + return code; + } + + public void setCode(Integer code) { + this.code = code; + } + + /** + **/ + public Error message(String message) { + this.message = message; + return this; + } + + @ApiModelProperty(example = "null", value = "") + @JsonProperty("message") + public String getMessage() { + return message; + } + + public void setMessage(String message) { + this.message = message; + } + + /** + **/ + public Error fields(String fields) { + this.fields = fields; + return this; + } + + @ApiModelProperty(example = "null", value = "") + @JsonProperty("fields") + public String getFields() { + return fields; + } + + public void setFields(String fields) { + this.fields = fields; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Error error = (Error) o; + return Objects.equals(this.code, error.code) + && Objects.equals(this.message, error.message) + && Objects.equals(this.fields, error.fields); + } + + @Override + public int hashCode() { + return Objects.hash(code, message, fields); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("class Error {\n"); + + sb.append(" code: ").append(toIndentedString(code)).append("\n"); + sb.append(" message: ").append(toIndentedString(message)).append("\n"); + sb.append(" fields: ").append(toIndentedString(fields)).append("\n"); + sb.append("}"); + return sb.toString(); + } + + /** + * Convert the given object to string with each line indented by 4 spaces + * (except the first line). + */ + private String toIndentedString(java.lang.Object o) { + if (o == null) { + return "null"; + } + return o.toString().replace("\n", "\n "); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/PlacementPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/PlacementPolicy.java new file mode 100644 index 0000000..6f6fe6f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/PlacementPolicy.java @@ -0,0 +1,102 @@ +/* + * 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.service.api.records; + +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; + +import java.io.Serializable; +import java.util.Objects; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Placement policy of an instance of an service. This feature is in the + * works in YARN-4902. + **/ +@InterfaceAudience.Public +@InterfaceStability.Unstable +@ApiModel(description = "Placement policy of an instance of an service. This feature is in the works in YARN-4902.") +@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") +public class PlacementPolicy implements Serializable { + private static final long serialVersionUID = 4341110649551172231L; + + private String label = null; + + /** + * Assigns a service to a named partition of the cluster where the service + * desires to run (optional). If not specified all services are submitted to + * a default label of the service owner. One or more labels can be setup for + * each service owner account with required constraints like no-preemption, + * sla-99999, preemption-ok, etc. + **/ + public PlacementPolicy label(String label) { + this.label = label; + return this; + } + + @ApiModelProperty(example = "null", value = "Assigns a service to a named partition of the cluster where the service desires to run (optional). If not specified all services are submitted to a default label of the service owner. One or more labels can be setup for each service owner account with required constraints like no-preemption, sla-99999, preemption-ok, etc.") + @JsonProperty("label") + public String getLabel() { + return label; + } + + public void setLabel(String label) { + this.label = label; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PlacementPolicy placementPolicy = (PlacementPolicy) o; + return Objects.equals(this.label, placementPolicy.label); + } + + @Override + public int hashCode() { + return Objects.hash(label); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("class PlacementPolicy {\n"); + + sb.append(" label: ").append(toIndentedString(label)).append("\n"); + sb.append("}"); + return sb.toString(); + } + + /** + * Convert the given object to string with each line indented by 4 spaces + * (except the first line). + */ + private String toIndentedString(java.lang.Object o) { + if (o == null) { + return "null"; + } + return o.toString().replace("\n", "\n "); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ReadinessCheck.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ReadinessCheck.java new file mode 100644 index 0000000..b25828f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ReadinessCheck.java @@ -0,0 +1,183 @@ +/* + * 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.service.api.records; + +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +import javax.xml.bind.annotation.XmlEnum; +import javax.xml.bind.annotation.XmlType; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonValue; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * A custom command or a pluggable helper container to determine the readiness + * of a container of a component. Readiness for every service is different. + * Hence the need for a simple interface, with scope to support advanced + * usecases. + **/ +@InterfaceAudience.Public +@InterfaceStability.Unstable +@ApiModel(description = "A custom command or a pluggable helper container to determine the readiness of a container of a component. Readiness for every service is different. Hence the need for a simple interface, with scope to support advanced usecases.") +@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") +public class ReadinessCheck implements Serializable { + private static final long serialVersionUID = -3836839816887186801L; + + /** + * Type. HTTP and PORT + **/ + @XmlType(name = "type") + @XmlEnum + public enum TypeEnum { + HTTP("HTTP"), + PORT("PORT"); + + private String value; + + TypeEnum(String value) { + this.value = value; + } + + @Override + @JsonValue + public String toString() { + return value; + } + } + + private TypeEnum type = null; + private Map props = new HashMap(); + private Artifact artifact = null; + + /** + * E.g. HTTP (YARN will perform a simple REST call at a regular interval and + * expect a 204 No content). + **/ + public ReadinessCheck type(TypeEnum type) { + this.type = type; + return this; + } + + @ApiModelProperty(example = "null", value = "E.g. HTTP (YARN will perform a simple REST call at a regular interval and expect a 204 No content).") + @JsonProperty("type") + public TypeEnum getType() { + return type; + } + + public void setType(TypeEnum type) { + this.type = type; + } + + public ReadinessCheck props(Map props) { + this.props = props; + return this; + } + + public ReadinessCheck putPropsItem(String key, String propsItem) { + this.props.put(key, propsItem); + return this; + } + + /** + * A blob of key value pairs that will be used to configure the check. + * @return props + **/ + @ApiModelProperty(example = "null", value = "A blob of key value pairs that will be used to configure the check.") + public Map getProps() { + return props; + } + + public void setProps(Map props) { + this.props = props; + } + + /** + * Artifact of the pluggable readiness check helper container (optional). If + * specified, this helper container typically hosts the http uri and + * encapsulates the complex scripts required to perform actual container + * readiness check. At the end it is expected to respond a 204 No content just + * like the simplified use case. This pluggable framework benefits service + * owners who can run services without any packaging modifications. Note, + * artifacts of type docker only is supported for now. + **/ + public ReadinessCheck artifact(Artifact artifact) { + this.artifact = artifact; + return this; + } + + @ApiModelProperty(example = "null", value = "Artifact of the pluggable readiness check helper container (optional). If specified, this helper container typically hosts the http uri and encapsulates the complex scripts required to perform actual container readiness check. At the end it is expected to respond a 204 No content just like the simplified use case. This pluggable framework benefits service owners who can run services without any packaging modifications. Note, artifacts of type docker only is supported for now.") + @JsonProperty("artifact") + public Artifact getArtifact() { + return artifact; + } + + public void setArtifact(Artifact artifact) { + this.artifact = artifact; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ReadinessCheck readinessCheck = (ReadinessCheck) o; + return Objects.equals(this.type, readinessCheck.type) && + Objects.equals(this.props, readinessCheck.props) && + Objects.equals(this.artifact, readinessCheck.artifact); + } + + @Override + public int hashCode() { + return Objects.hash(type, props, artifact); + } + + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("class ReadinessCheck {\n"); + + sb.append(" type: ").append(toIndentedString(type)).append("\n"); + sb.append(" props: ").append(toIndentedString(props)).append("\n"); + sb.append(" artifact: ").append(toIndentedString(artifact)).append("\n"); + sb.append("}"); + return sb.toString(); + } + + /** + * Convert the given object to string with each line indented by 4 spaces + * (except the first line). + */ + private String toIndentedString(java.lang.Object o) { + if (o == null) { + return "null"; + } + return o.toString().replace("\n", "\n "); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Resource.java new file mode 100644 index 0000000..dfdf92a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Resource.java @@ -0,0 +1,161 @@ +/* + * 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.service.api.records; + +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; + +import java.util.Objects; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Resource determines the amount of resources (vcores, memory, network, etc.) + * usable by a container. This field determines the resource to be applied for + * all the containers of a component or service. The resource specified at + * the service (or global) level can be overriden at the component level. Only one + * of profile OR cpu & memory are expected. It raises a validation + * exception otherwise. + **/ +@InterfaceAudience.Public +@InterfaceStability.Unstable +@ApiModel(description = "Resource determines the amount of resources (vcores, memory, network, etc.) usable by a container. This field determines the resource to be applied for all the containers of a component or service. The resource specified at the service (or global) level can be overriden at the component level. Only one of profile OR cpu & memory are expected. It raises a validation exception otherwise.") +@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") +public class Resource extends BaseResource implements Cloneable { + private static final long serialVersionUID = -6431667797380250037L; + + private String profile = null; + private Integer cpus = 1; + private String memory = null; + + /** + * Each resource profile has a unique id which is associated with a + * cluster-level predefined memory, cpus, etc. + **/ + public Resource profile(String profile) { + this.profile = profile; + return this; + } + + @ApiModelProperty(example = "null", value = "Each resource profile has a unique id which is associated with a cluster-level predefined memory, cpus, etc.") + @JsonProperty("profile") + public String getProfile() { + return profile; + } + + public void setProfile(String profile) { + this.profile = profile; + } + + /** + * Amount of vcores allocated to each container (optional but overrides cpus + * in profile if specified). + **/ + public Resource cpus(Integer cpus) { + this.cpus = cpus; + return this; + } + + @ApiModelProperty(example = "null", value = "Amount of vcores allocated to each container (optional but overrides cpus in profile if specified).") + @JsonProperty("cpus") + public Integer getCpus() { + return cpus; + } + + public void setCpus(Integer cpus) { + this.cpus = cpus; + } + + /** + * Amount of memory allocated to each container (optional but overrides memory + * in profile if specified). Currently accepts only an integer value and + * default unit is in MB. + **/ + public Resource memory(String memory) { + this.memory = memory; + return this; + } + + @ApiModelProperty(example = "null", value = "Amount of memory allocated to each container (optional but overrides memory in profile if specified). Currently accepts only an integer value and default unit is in MB.") + @JsonProperty("memory") + public String getMemory() { + return memory; + } + + public void setMemory(String memory) { + this.memory = memory; + } + + @JsonIgnore + public long getMemoryMB() { + if (this.memory == null) { + return 0; + } + return Long.parseLong(memory); + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Resource resource = (Resource) o; + return Objects.equals(this.profile, resource.profile) + && Objects.equals(this.cpus, resource.cpus) + && Objects.equals(this.memory, resource.memory); + } + + @Override + public int hashCode() { + return Objects.hash(profile, cpus, memory); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("class Resource {\n"); + + sb.append(" profile: ").append(toIndentedString(profile)).append("\n"); + sb.append(" cpus: ").append(toIndentedString(cpus)).append("\n"); + sb.append(" memory: ").append(toIndentedString(memory)).append("\n"); + sb.append("}"); + return sb.toString(); + } + + /** + * Convert the given object to string with each line indented by 4 spaces + * (except the first line). + */ + private String toIndentedString(java.lang.Object o) { + if (o == null) { + return "null"; + } + return o.toString().replace("\n", "\n "); + } + + @Override + public Object clone() throws CloneNotSupportedException { + return super.clone(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Service.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Service.java new file mode 100644 index 0000000..f3bbaa0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Service.java @@ -0,0 +1,466 @@ +/* + * 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.service.api.records; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import javax.xml.bind.annotation.XmlElement; +import javax.xml.bind.annotation.XmlRootElement; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * An Service resource has the following attributes. + **/ +@InterfaceAudience.Public +@InterfaceStability.Unstable +@ApiModel(description = "An Service resource has the following attributes.") +@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") +@XmlRootElement +@JsonInclude(JsonInclude.Include.NON_NULL) +@JsonPropertyOrder({ "name", "state", "resource", "number_of_containers", + "lifetime", "containers" }) +public class Service extends BaseResource { + private static final long serialVersionUID = -4491694636566094885L; + + private String name = null; + private String id = null; + private Artifact artifact = null; + private Resource resource = null; + private String launchCommand = null; + private Date launchTime = null; + private Long numberOfContainers = null; + private Long numberOfRunningContainers = null; + private Long lifetime = null; + private PlacementPolicy placementPolicy = null; + private List components = new ArrayList<>(); + private Configuration configuration = new Configuration(); + private List containers = new ArrayList<>(); + private ServiceState state = null; + private Map quicklinks = new HashMap<>(); + private String queue = null; + + /** + * A unique service name. + **/ + public Service name(String name) { + this.name = name; + return this; + } + + @ApiModelProperty(example = "null", required = true, value = "A unique service name.") + @JsonProperty("name") + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * A unique service id. + **/ + public Service id(String id) { + this.id = id; + return this; + } + + @ApiModelProperty(example = "null", value = "A unique service id.") + @JsonProperty("id") + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * Artifact of single-component services. Mandatory if components + * attribute is not specified. + **/ + public Service artifact(Artifact artifact) { + this.artifact = artifact; + return this; + } + + @ApiModelProperty(example = "null", value = "Artifact of single-component services. Mandatory if components attribute is not specified.") + @JsonProperty("artifact") + public Artifact getArtifact() { + return artifact; + } + + public void setArtifact(Artifact artifact) { + this.artifact = artifact; + } + + /** + * Resource of single-component services or the global default for + * multi-component services. Mandatory if it is a single-component + * service and if cpus and memory are not specified at the Service + * level. + **/ + public Service resource(Resource resource) { + this.resource = resource; + return this; + } + + @ApiModelProperty(example = "null", value = "Resource of single-component services or the global default for multi-component services. Mandatory if it is a single-component service and if cpus and memory are not specified at the Service level.") + @JsonProperty("resource") + public Resource getResource() { + return resource; + } + + public void setResource(Resource resource) { + this.resource = resource; + } + + /** + * The custom launch command of an service component (optional). If not + * specified for services with docker images say, it will default to the + * default start command of the image. If there is a single component in this + * service, you can specify this without the need to have a 'components' + * section. + **/ + public Service launchCommand(String launchCommand) { + this.launchCommand = launchCommand; + return this; + } + + @ApiModelProperty(example = "null", value = "The custom launch command of an service component (optional). If not specified for services with docker images say, it will default to the default start command of the image. If there is a single component in this service, you can specify this without the need to have a 'components' section.") + @JsonProperty("launch_command") + public String getLaunchCommand() { + return launchCommand; + } + + @XmlElement(name = "launch_command") + public void setLaunchCommand(String launchCommand) { + this.launchCommand = launchCommand; + } + + /** + * The time when the service was created, e.g. 2016-03-16T01:01:49.000Z. + **/ + public Service launchTime(Date launchTime) { + this.launchTime = launchTime == null ? null : (Date) launchTime.clone(); + return this; + } + + @ApiModelProperty(example = "null", value = "The time when the service was created, e.g. 2016-03-16T01:01:49.000Z.") + @JsonProperty("launch_time") + public Date getLaunchTime() { + return launchTime == null ? null : (Date) launchTime.clone(); + } + + @XmlElement(name = "launch_time") + public void setLaunchTime(Date launchTime) { + this.launchTime = launchTime == null ? null : (Date) launchTime.clone(); + } + + /** + * Number of containers for each component in the service. Each + * component can further override this service-level global default. + **/ + public Service numberOfContainers(Long numberOfContainers) { + this.numberOfContainers = numberOfContainers; + return this; + } + + @ApiModelProperty(example = "null", value = "Number of containers for each component in the service. Each component can further override this service-level global default.") + @JsonProperty("number_of_containers") + public Long getNumberOfContainers() { + return numberOfContainers; + } + + @XmlElement(name = "number_of_containers") + public void setNumberOfContainers(Long numberOfContainers) { + this.numberOfContainers = numberOfContainers; + } + + /** + * In get response this provides the total number of running containers for + * this service (across all components) at the time of request. Note, a + * subsequent request can return a different number as and when more + * containers get allocated until it reaches the total number of containers or + * if a flex request has been made between the two requests. + **/ + public Service numberOfRunningContainers(Long numberOfRunningContainers) { + this.numberOfRunningContainers = numberOfRunningContainers; + return this; + } + + @ApiModelProperty(example = "null", value = "In get response this provides the total number of running containers for this service (across all components) at the time of request. Note, a subsequent request can return a different number as and when more containers get allocated until it reaches the total number of containers or if a flex request has been made between the two requests.") + @JsonProperty("number_of_running_containers") + public Long getNumberOfRunningContainers() { + return numberOfRunningContainers; + } + + @XmlElement(name = "number_of_running_containers") + public void setNumberOfRunningContainers(Long numberOfRunningContainers) { + this.numberOfRunningContainers = numberOfRunningContainers; + } + + /** + * Life time (in seconds) of the service from the time it reaches the + * RUNNING_BUT_UNREADY state (after which it is automatically destroyed by YARN). For + * unlimited lifetime do not set a lifetime value. + **/ + public Service lifetime(Long lifetime) { + this.lifetime = lifetime; + return this; + } + + @ApiModelProperty(example = "null", value = "Life time (in seconds) of the service from the time it reaches the RUNNING_BUT_UNREADY state (after which it is automatically destroyed by YARN). For unlimited lifetime do not set a lifetime value.") + @JsonProperty("lifetime") + public Long getLifetime() { + return lifetime; + } + + public void setLifetime(Long lifetime) { + this.lifetime = lifetime; + } + + /** + * Advanced scheduling and placement policies (optional). If not specified, it + * defaults to the default placement policy of the service owner. The design of + * placement policies are in the works. It is not very clear at this point, + * how policies in conjunction with labels be exposed to service owners. + * This is a placeholder for now. The advanced structure of this attribute + * will be determined by YARN-4902. + **/ + public Service placementPolicy(PlacementPolicy placementPolicy) { + this.placementPolicy = placementPolicy; + return this; + } + + @ApiModelProperty(example = "null", value = "Advanced scheduling and placement policies (optional). If not specified, it defaults to the default placement policy of the service owner. The design of placement policies are in the works. It is not very clear at this point, how policies in conjunction with labels be exposed to service owners. This is a placeholder for now. The advanced structure of this attribute will be determined by YARN-4902.") + @JsonProperty("placement_policy") + public PlacementPolicy getPlacementPolicy() { + return placementPolicy; + } + + @XmlElement(name = "placement_policy") + public void setPlacementPolicy(PlacementPolicy placementPolicy) { + this.placementPolicy = placementPolicy; + } + + /** + * Components of an service. + **/ + public Service components(List components) { + this.components = components; + return this; + } + + @ApiModelProperty(example = "null", value = "Components of an service.") + @JsonProperty("components") + public List getComponents() { + return components; + } + + public void setComponents(List components) { + this.components = components; + } + + public void addComponent(Component component) { + components.add(component); + } + + public Component getComponent(String name) { + for (Component component : components) { + if (component.getName().equals(name)) { + return component; + } + } + return null; + } + + /** + * Config properties of an service. Configurations provided at the + * service/global level are available to all the components. Specific + * properties can be overridden at the component level. + **/ + public Service configuration(Configuration configuration) { + this.configuration = configuration; + return this; + } + + @ApiModelProperty(example = "null", value = "Config properties of an service. Configurations provided at the service/global level are available to all the components. Specific properties can be overridden at the component level.") + @JsonProperty("configuration") + public Configuration getConfiguration() { + return configuration; + } + + public void setConfiguration(Configuration configuration) { + this.configuration = configuration; + } + + /** + * Containers of a started service. Specifying a value for this attribute + * for the POST payload raises a validation error. This blob is available only + * in the GET response of a started service. + **/ + public Service containers(List containers) { + this.containers = containers; + return this; + } + + @ApiModelProperty(example = "null", value = "Containers of a started service. Specifying a value for this attribute for the POST payload raises a validation error. This blob is available only in the GET response of a started service.") + @JsonProperty("containers") + public List getContainers() { + return containers; + } + + public void setContainers(List containers) { + this.containers = containers; + } + + public void addContainer(Container container) { + this.containers.add(container); + } + + /** + * State of the service. Specifying a value for this attribute for the + * POST payload raises a validation error. This attribute is available only in + * the GET response of a started service. + **/ + public Service state(ServiceState state) { + this.state = state; + return this; + } + + @ApiModelProperty(example = "null", value = "State of the service. Specifying a value for this attribute for the POST payload raises a validation error. This attribute is available only in the GET response of a started service.") + @JsonProperty("state") + public ServiceState getState() { + return state; + } + + public void setState(ServiceState state) { + this.state = state; + } + + /** + * A blob of key-value pairs of quicklinks to be exported for an service. + **/ + public Service quicklinks(Map quicklinks) { + this.quicklinks = quicklinks; + return this; + } + + @ApiModelProperty(example = "null", value = "A blob of key-value pairs of quicklinks to be exported for an service.") + @JsonProperty("quicklinks") + public Map getQuicklinks() { + return quicklinks; + } + + public void setQuicklinks(Map quicklinks) { + this.quicklinks = quicklinks; + } + + /** + * The YARN queue that this service should be submitted to. + **/ + public Service queue(String queue) { + this.queue = queue; + return this; + } + + @ApiModelProperty(example = "null", value = "The YARN queue that this service should be submitted to.") + @JsonProperty("queue") + public String getQueue() { + return queue; + } + + public void setQueue(String queue) { + this.queue = queue; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Service service = (Service) o; + return Objects.equals(this.name, service.name); + } + + @Override + public int hashCode() { + return Objects.hash(name); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("class Service {\n"); + + sb.append(" name: ").append(toIndentedString(name)).append("\n"); + sb.append(" id: ").append(toIndentedString(id)).append("\n"); + sb.append(" artifact: ").append(toIndentedString(artifact)).append("\n"); + sb.append(" resource: ").append(toIndentedString(resource)).append("\n"); + sb.append(" launchCommand: ").append(toIndentedString(launchCommand)) + .append("\n"); + sb.append(" launchTime: ").append(toIndentedString(launchTime)) + .append("\n"); + sb.append(" numberOfContainers: ") + .append(toIndentedString(numberOfContainers)).append("\n"); + sb.append(" numberOfRunningContainers: ") + .append(toIndentedString(numberOfRunningContainers)).append("\n"); + sb.append(" lifetime: ").append(toIndentedString(lifetime)).append("\n"); + sb.append(" placementPolicy: ").append(toIndentedString(placementPolicy)) + .append("\n"); + sb.append(" components: ").append(toIndentedString(components)) + .append("\n"); + sb.append(" configuration: ").append(toIndentedString(configuration)) + .append("\n"); + sb.append(" containers: ").append(toIndentedString(containers)) + .append("\n"); + sb.append(" state: ").append(toIndentedString(state)).append("\n"); + sb.append(" quicklinks: ").append(toIndentedString(quicklinks)) + .append("\n"); + sb.append(" queue: ").append(toIndentedString(queue)).append("\n"); + sb.append("}"); + return sb.toString(); + } + + /** + * Convert the given object to string with each line indented by 4 spaces + * (except the first line). + */ + private String toIndentedString(java.lang.Object o) { + if (o == null) { + return "null"; + } + return o.toString().replace("\n", "\n "); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java new file mode 100644 index 0000000..a4509bd --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java @@ -0,0 +1,33 @@ +/* + * 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.service.api.records; + +import io.swagger.annotations.ApiModel; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * The current state of an service. + **/ +@InterfaceAudience.Public +@InterfaceStability.Unstable +@ApiModel(description = "The current state of an service.") +@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") +public enum ServiceState { + ACCEPTED, STARTED, READY, STOPPED, FAILED; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceStatus.java new file mode 100644 index 0000000..2cee23c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceStatus.java @@ -0,0 +1,148 @@ +/* + * 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.service.api.records; + +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; + +import java.util.Objects; + +import javax.xml.bind.annotation.XmlRootElement; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * The current status of a submitted service, returned as a response to the + * GET API. + **/ +@InterfaceAudience.Public +@InterfaceStability.Unstable +@ApiModel(description = "The current status of a submitted service, returned as a response to the GET API.") +@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") +@XmlRootElement +@JsonInclude(JsonInclude.Include.NON_NULL) +public class ServiceStatus extends BaseResource { + private static final long serialVersionUID = -3469885905347851034L; + + private String diagnostics = null; + private ServiceState state = null; + private Integer code = null; + + /** + * Diagnostic information (if any) for the reason of the current state of the + * service. It typically has a non-null value, if the service is in a + * non-running state. + **/ + public ServiceStatus diagnostics(String diagnostics) { + this.diagnostics = diagnostics; + return this; + } + + @ApiModelProperty(example = "null", value = "Diagnostic information (if any) for the reason of the current state of the service. It typically has a non-null value, if the service is in a non-running state.") + @JsonProperty("diagnostics") + public String getDiagnostics() { + return diagnostics; + } + + public void setDiagnostics(String diagnostics) { + this.diagnostics = diagnostics; + } + + /** + * Service state. + **/ + public ServiceStatus state(ServiceState state) { + this.state = state; + return this; + } + + @ApiModelProperty(example = "null", value = "Service state.") + @JsonProperty("state") + public ServiceState getState() { + return state; + } + + public void setState(ServiceState state) { + this.state = state; + } + + /** + * An error code specific to a scenario which service owners should be able to use + * to understand the failure in addition to the diagnostic information. + **/ + public ServiceStatus code(Integer code) { + this.code = code; + return this; + } + + @ApiModelProperty(example = "null", value = "An error code specific to a scenario which service owners should be able to use to understand the failure in addition to the diagnostic information.") + @JsonProperty("code") + public Integer getCode() { + return code; + } + + public void setCode(Integer code) { + this.code = code; + } + + @Override + public boolean equals(java.lang.Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ServiceStatus serviceStatus = (ServiceStatus) o; + return Objects.equals(this.diagnostics, serviceStatus.diagnostics) + && Objects.equals(this.state, serviceStatus.state) + && Objects.equals(this.code, serviceStatus.code); + } + + @Override + public int hashCode() { + return Objects.hash(diagnostics, state, code); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("class ServiceStatus {\n"); + + sb.append(" diagnostics: ").append(toIndentedString(diagnostics)) + .append("\n"); + sb.append(" state: ").append(toIndentedString(state)).append("\n"); + sb.append(" code: ").append(toIndentedString(code)).append("\n"); + sb.append("}"); + return sb.toString(); + } + + /** + * Convert the given object to string with each line indented by 4 spaces + * (except the first line). + */ + private String toIndentedString(java.lang.Object o) { + if (o == null) { + return "null"; + } + return o.toString().replace("\n", "\n "); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ClientAMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ClientAMProxy.java new file mode 100644 index 0000000..e17c0c4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ClientAMProxy.java @@ -0,0 +1,57 @@ +/** + * 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.service.client; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.client.ServerProxy; +import org.apache.hadoop.yarn.ipc.YarnRPC; +import org.apache.hadoop.yarn.service.conf.YarnServiceConf; + +import java.net.InetSocketAddress; + +import static org.apache.hadoop.io.retry.RetryPolicies.TRY_ONCE_THEN_FAIL; + +public class ClientAMProxy extends ServerProxy{ + + public static T createProxy(final Configuration conf, + final Class protocol, final UserGroupInformation ugi, + final YarnRPC rpc, final InetSocketAddress serverAddress) { + Configuration confClone = new Configuration(conf); + confClone.setInt( + CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0); + confClone.setInt(CommonConfigurationKeysPublic. + IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0); + RetryPolicy retryPolicy; + + if (conf.getLong(YarnServiceConf.CLIENT_AM_RETRY_MAX_WAIT_MS, 0) == 0) { + // by default no retry + retryPolicy = TRY_ONCE_THEN_FAIL; + } else { + retryPolicy = + createRetryPolicy(conf, YarnServiceConf.CLIENT_AM_RETRY_MAX_WAIT_MS, + 15 * 60 * 1000, YarnServiceConf.CLIENT_AM_RETRY_MAX_INTERVAL_MS, + 2 * 1000); + } + return createRetriableProxy(confClone, protocol, ugi, rpc, serverAddress, + retryPolicy); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceCLI.java new file mode 100644 index 0000000..928c06f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceCLI.java @@ -0,0 +1,113 @@ +/** + * 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.service.client; + +import com.beust.jcommander.ParameterException; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.service.api.records.Service; +import org.apache.hadoop.yarn.service.client.params.ClientArgs; +import org.apache.hadoop.yarn.service.exceptions.SliderException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; + +import static org.apache.hadoop.yarn.service.client.params.SliderActions.*; + +public class ServiceCLI { + private static final Logger LOG = + LoggerFactory.getLogger(ServiceClient.class); + protected ServiceClient client; + + int exec(ClientArgs args) throws Throwable { + if (StringUtils.isEmpty(args.getAction())) { + System.out.println(args.usage()); + return -1; + } + switch (args.getAction()) { + case ACTION_BUILD: // Upload app json onto hdfs + client.actionBuild(args.getActionBuildArgs()); + break; + case ACTION_START: // start the app with the pre-uploaded app json on hdfs + client.actionStart(args.getClusterName()); + break; + case ACTION_CREATE: // create == build + start + client.actionCreate(args.getActionCreateArgs()); + break; + case ACTION_STATUS: + Service app = client.getStatus(args.getClusterName()); + System.out.println(app); + break; + case ACTION_FLEX: + try { + client.actionFlexByCLI(args); + } catch (FileNotFoundException e) { + System.err.println( + args.getClusterName() + " doesn't exist: " + e.getMessage()); + return -1; + } + break; + case ACTION_STOP: + client.actionStop(args.getClusterName(), false); + break; + case ACTION_DESTROY: // Destroy can happen only if app is already stopped + client.actionDestroy(args.getClusterName()); + break; + case ACTION_DEPENDENCY: // upload dependency jars + client.actionDependency(args.getActionDependencyArgs()); + break; + case ACTION_UPDATE: + client.updateLifetime(args.getClusterName(), + args.getActionUpdateArgs().lifetime); + break; + case ACTION_HELP: + LOG.info(args.usage()); + break; + default: + LOG.info("NOT IMPLEMENTED: " + args.getAction()); + LOG.info(args.usage()); + return -1; + } + return 0; + } + + public ServiceCLI() { + createServiceClient(); + } + + protected void createServiceClient() { + client = new ServiceClient(); + client.init(new YarnConfiguration()); + client.start(); + } + + public static void main(String[] args) throws Throwable { + ClientArgs clientArgs = new ClientArgs(args); + try { + clientArgs.parse(); + } catch (ParameterException | SliderException e) { + System.err.println(e.getMessage()); + System.exit(-1); + } + ServiceCLI cli = new ServiceCLI(); + int res = cli.exec(clientArgs); + System.exit(res); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java new file mode 100644 index 0000000..a3a9fd0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java @@ -0,0 +1,919 @@ +/** + * 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.service.client; + +import org.apache.commons.lang.StringUtils; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.RetryNTimes; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.registry.client.api.RegistryConstants; +import org.apache.hadoop.registry.client.api.RegistryOperations; +import org.apache.hadoop.registry.client.api.RegistryOperationsFactory; +import org.apache.hadoop.registry.client.binding.RegistryUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.util.VersionInfo; +import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.ApplicationTimeout; +import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.hadoop.yarn.api.records.LocalResourceType; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.client.api.YarnClientApplication; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.ipc.YarnRPC; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.ComponentCountProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto; +import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopRequestProto; +import org.apache.hadoop.yarn.service.ClientAMProtocol; +import org.apache.hadoop.yarn.service.ServiceMaster; +import org.apache.hadoop.yarn.service.api.records.Service; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.api.records.ServiceState; +import org.apache.hadoop.yarn.service.client.params.AbstractClusterBuildingActionArgs; +import org.apache.hadoop.yarn.service.client.params.ActionCreateArgs; +import org.apache.hadoop.yarn.service.client.params.ActionDependencyArgs; +import org.apache.hadoop.yarn.service.client.params.ActionFlexArgs; +import org.apache.hadoop.yarn.service.client.params.Arguments; +import org.apache.hadoop.yarn.service.client.params.ClientArgs; +import org.apache.hadoop.yarn.service.client.params.CommonArgs; +import org.apache.hadoop.yarn.service.conf.SliderExitCodes; +import org.apache.hadoop.yarn.service.conf.YarnServiceConstants; +import org.apache.hadoop.yarn.service.conf.YarnServiceConf; +import org.apache.hadoop.yarn.service.provider.AbstractClientProvider; +import org.apache.hadoop.yarn.service.provider.ProviderUtils; +import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; +import org.apache.hadoop.yarn.service.utils.ServiceRegistryUtils; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; +import org.apache.hadoop.yarn.service.utils.SliderUtils; +import org.apache.hadoop.yarn.util.Records; +import org.apache.hadoop.yarn.util.Times; +import org.apache.hadoop.yarn.service.exceptions.BadClusterStateException; +import org.apache.hadoop.yarn.service.exceptions.BadConfigException; +import org.apache.hadoop.yarn.service.exceptions.SliderException; +import org.apache.hadoop.yarn.service.exceptions.UsageException; +import org.apache.hadoop.yarn.service.containerlaunch.ClasspathConstructor; +import org.apache.hadoop.yarn.service.containerlaunch.JavaCommandLineBuilder; +import org.apache.hadoop.yarn.service.utils.ZookeeperUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.text.MessageFormat; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import static org.apache.hadoop.yarn.api.records.YarnApplicationState.*; +import static org.apache.hadoop.yarn.service.client.params.SliderActions.ACTION_CREATE; +import static org.apache.hadoop.yarn.service.client.params.SliderActions.ACTION_FLEX; +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.YARN_QUEUE; +import static org.apache.hadoop.yarn.service.utils.ServiceApiUtil.jsonSerDeser; +import static org.apache.hadoop.yarn.service.utils.SliderUtils.*; + +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class ServiceClient extends CompositeService + implements SliderExitCodes, YarnServiceConstants { + private static final Logger LOG = + LoggerFactory.getLogger(ServiceClient.class); + private SliderFileSystem fs; + //TODO disable retry so that client / rest API doesn't block? + protected YarnClient yarnClient; + // Avoid looking up applicationId from fs all the time. + private Map cachedAppIds = new ConcurrentHashMap<>(); + + private RegistryOperations registryClient; + private CuratorFramework curatorClient; + private YarnRPC rpc; + + private static EnumSet terminatedStates = + EnumSet.of(FINISHED, FAILED, KILLED); + private static EnumSet liveStates = + EnumSet.of(NEW, NEW_SAVING, SUBMITTED, ACCEPTED, RUNNING); + private static EnumSet preRunningStates = + EnumSet.of(NEW, NEW_SAVING, SUBMITTED, ACCEPTED); + + public ServiceClient() { + super(ServiceClient.class.getName()); + } + + @Override protected void serviceInit(Configuration configuration) + throws Exception { + fs = new SliderFileSystem(configuration); + yarnClient = YarnClient.createYarnClient(); + rpc = YarnRPC.create(configuration); + addService(yarnClient); + super.serviceInit(configuration); + } + + @Override + protected void serviceStop() throws Exception { + if (registryClient != null) { + registryClient.stop(); + } + super.serviceStop(); + } + + private Service loadAppJsonFromLocalFS( + AbstractClusterBuildingActionArgs args) throws IOException { + File file = args.getFile(); + Path filePath = new Path(file.getAbsolutePath()); + LOG.info("Loading service definition from: " + filePath); + Service service = jsonSerDeser + .load(FileSystem.getLocal(getConfig()), filePath); + if (args.lifetime > 0) { + service.setLifetime(args.lifetime); + } + if (!StringUtils.isEmpty(args.getServiceName())) { + service.setName(args.getServiceName()); + } + if (!StringUtils.isEmpty(args.queue)) { + service.setQueue(args.queue); + } + return service; + } + + public int actionBuild(AbstractClusterBuildingActionArgs args) + throws IOException, YarnException { + return actionBuild(loadAppJsonFromLocalFS(args)); + } + + public int actionBuild(Service service) + throws YarnException, IOException { + Path appDir = checkAppNotExistOnHdfs(service); + ServiceApiUtil.validateAndResolveService(service, fs, getConfig()); + createDirAndPersistApp(appDir, service); + return EXIT_SUCCESS; + } + + public int actionCreate(ActionCreateArgs args) + throws IOException, YarnException { + Service serviceDef; + if (args.file != null) { + serviceDef = loadAppJsonFromLocalFS(args); + } else if (!StringUtils.isEmpty(args.example)) { + // create an example service + String yarnHome = System + .getenv(ApplicationConstants.Environment.HADOOP_YARN_HOME.key()); + args.file = new File(MessageFormat + .format("{0}/share/hadoop/yarn/yarn-service-examples/{1}/{2}.json", + yarnHome, args.example, args.example)); + serviceDef = loadAppJsonFromLocalFS(args); + } else { + throw new YarnException("No service definition provided!"); + } + actionCreate(serviceDef); + return EXIT_SUCCESS; + } + + public ApplicationId actionCreate(Service service) + throws IOException, YarnException { + String serviceName = service.getName(); + ServiceApiUtil.validateNameFormat(serviceName, getConfig()); + ServiceApiUtil.validateAndResolveService(service, fs, getConfig()); + verifyNoLiveAppInRM(serviceName, "create"); + Path appDir = checkAppNotExistOnHdfs(service); + + // Write the definition first and then submit - AM will read the definition + createDirAndPersistApp(appDir, service); + ApplicationId appId = submitApp(service); + cachedAppIds.put(serviceName, appId); + service.setId(appId.toString()); + // update app definition with appId + persistAppDef(appDir, service); + return appId; + } + + // Called by ServiceCLI + protected int actionFlexByCLI(ClientArgs args) + throws YarnException, IOException { + ActionFlexArgs flexArgs = args.getActionFlexArgs(); + Map componentCounts = + new HashMap<>(flexArgs.getComponentMap().size()); + Service persistedService = + ServiceApiUtil.loadService(fs, flexArgs.getServiceName()); + if (!StringUtils.isEmpty(persistedService.getId())) { + cachedAppIds.put(persistedService.getName(), + ApplicationId.fromString(persistedService.getId())); + } else { + throw new YarnException(persistedService.getName() + + " appId is null, may be not submitted to YARN yet"); + } + + for (Map.Entry entry : flexArgs.getComponentMap() + .entrySet()) { + String compName = entry.getKey(); + ServiceApiUtil.validateNameFormat(compName, getConfig()); + Component component = persistedService.getComponent(compName); + if (component == null) { + throw new IllegalArgumentException(entry.getKey() + " does not exist !"); + } + long numberOfContainers = + parseNumberOfContainers(component, entry.getValue()); + componentCounts.put(compName, numberOfContainers); + } + // throw usage exception if no changes proposed + if (componentCounts.size() == 0) { + actionHelp(ACTION_FLEX, args); + } + flexComponents(args.getClusterName(), componentCounts, persistedService); + return EXIT_SUCCESS; + } + + // Parse the number of containers requested by user, e.g. + // +5 means add 5 additional containers + // -5 means reduce 5 containers, if it goes to negative, sets it to 0 + // 5 means sets it to 5 containers. + private long parseNumberOfContainers(Component component, String newNumber) { + + long orig = component.getNumberOfContainers(); + if (newNumber.startsWith("+")) { + return orig + Long.parseLong(newNumber.substring(1)); + } else if (newNumber.startsWith("-")) { + long ret = orig - Long.parseLong(newNumber.substring(1)); + if (ret < 0) { + LOG.warn(MessageFormat.format( + "[COMPONENT {}]: component count goes to negative ({}{} = {}), reset it to 0.", + component.getName(), orig, newNumber, ret)); + ret = 0; + } + return ret; + } else { + return Long.parseLong(newNumber); + } + } + + // Called by Rest Service + public Map flexByRestService(String serviceName, + Map componentCounts) throws YarnException, IOException { + // load app definition + Service persistedService = ServiceApiUtil.loadService(fs, serviceName); + if (StringUtils.isEmpty(persistedService.getId())) { + throw new YarnException( + serviceName + " appId is null, may be not submitted to YARN yet"); + } + cachedAppIds.put(persistedService.getName(), + ApplicationId.fromString(persistedService.getId())); + return flexComponents(serviceName, componentCounts, persistedService); + } + + private Map flexComponents(String serviceName, + Map componentCounts, Service persistedService) + throws YarnException, IOException { + ServiceApiUtil.validateNameFormat(serviceName, getConfig()); + + Map original = new HashMap<>(componentCounts.size()); + + ComponentCountProto.Builder countBuilder = ComponentCountProto.newBuilder(); + FlexComponentsRequestProto.Builder requestBuilder = + FlexComponentsRequestProto.newBuilder(); + + for (Component persistedComp : persistedService.getComponents()) { + String name = persistedComp.getName(); + if (componentCounts.containsKey(persistedComp.getName())) { + original.put(name, persistedComp.getNumberOfContainers()); + persistedComp.setNumberOfContainers(componentCounts.get(name)); + + // build the request + countBuilder.setName(persistedComp.getName()) + .setNumberOfContainers(persistedComp.getNumberOfContainers()); + requestBuilder.addComponents(countBuilder.build()); + } + } + if (original.size() < componentCounts.size()) { + componentCounts.keySet().removeAll(original.keySet()); + throw new YarnException("Components " + componentCounts.keySet() + + " do not exist in app definition."); + } + jsonSerDeser + .save(fs.getFileSystem(), ServiceApiUtil.getServiceJsonPath(fs, serviceName), + persistedService, true); + + ApplicationReport appReport = + yarnClient.getApplicationReport(getAppId(serviceName)); + if (appReport.getYarnApplicationState() != RUNNING) { + String message = + serviceName + " is at " + appReport.getYarnApplicationState() + + " state, flex can only be invoked when service is running"; + LOG.error(message); + throw new YarnException(message); + } + if (StringUtils.isEmpty(appReport.getHost())) { + throw new YarnException(serviceName + " AM hostname is empty"); + } + ClientAMProtocol proxy = + createAMProxy(appReport.getHost(), appReport.getRpcPort()); + proxy.flexComponents(requestBuilder.build()); + for (Map.Entry entry : original.entrySet()) { + LOG.info("[COMPONENT {}]: number of containers changed from {} to {}", + entry.getKey(), entry.getValue(), + componentCounts.get(entry.getKey())); + } + return original; + } + + public int actionStop(String serviceName, boolean waitForAppStopped) + throws YarnException, IOException { + ServiceApiUtil.validateNameFormat(serviceName, getConfig()); + ApplicationId currentAppId = getAppId(serviceName); + ApplicationReport report = yarnClient.getApplicationReport(currentAppId); + if (terminatedStates.contains(report.getYarnApplicationState())) { + LOG.info("Service {} is already in a terminated state {}", serviceName, + report.getYarnApplicationState()); + return EXIT_SUCCESS; + } + if (preRunningStates.contains(report.getYarnApplicationState())) { + String msg = serviceName + " is at " + report.getYarnApplicationState() + + ", forcefully killed by user!"; + yarnClient.killApplication(currentAppId, msg); + LOG.info(msg); + return EXIT_SUCCESS; + } + if (StringUtils.isEmpty(report.getHost())) { + throw new YarnException(serviceName + " AM hostname is empty"); + } + LOG.info("Stopping service {}, with appId = {}", serviceName, currentAppId); + try { + ClientAMProtocol proxy = + createAMProxy(report.getHost(), report.getRpcPort()); + cachedAppIds.remove(serviceName); + if (proxy != null) { + // try to stop the app gracefully. + StopRequestProto request = StopRequestProto.newBuilder().build(); + proxy.stop(request); + LOG.info("Service " + serviceName + " is being gracefully stopped..."); + } else { + yarnClient.killApplication(currentAppId, + serviceName + " is forcefully killed by user!"); + LOG.info("Forcefully kill the service: " + serviceName); + return EXIT_SUCCESS; + } + + if (!waitForAppStopped) { + return EXIT_SUCCESS; + } + // Wait until the app is killed. + long startTime = System.currentTimeMillis(); + int pollCount = 0; + while (true) { + Thread.sleep(2000); + report = yarnClient.getApplicationReport(currentAppId); + if (terminatedStates.contains(report.getYarnApplicationState())) { + LOG.info("Service " + serviceName + " is stopped."); + break; + } + // Forcefully kill after 10 seconds. + if ((System.currentTimeMillis() - startTime) > 10000) { + LOG.info("Stop operation timeout stopping, forcefully kill the app " + + serviceName); + yarnClient.killApplication(currentAppId, + "Forcefully kill the app by user"); + break; + } + if (++pollCount % 10 == 0) { + LOG.info("Waiting for service " + serviceName + " to be stopped."); + } + } + } catch (IOException | YarnException | InterruptedException e) { + LOG.info("Failed to stop " + serviceName + + " gracefully, forcefully kill the app."); + yarnClient.killApplication(currentAppId, "Forcefully kill the app"); + } + return EXIT_SUCCESS; + } + + public int actionDestroy(String serviceName) throws Exception { + ServiceApiUtil.validateNameFormat(serviceName, getConfig()); + verifyNoLiveAppInRM(serviceName, "destroy"); + + Path appDir = fs.buildClusterDirPath(serviceName); + FileSystem fileSystem = fs.getFileSystem(); + // remove from the appId cache + cachedAppIds.remove(serviceName); + if (fileSystem.exists(appDir)) { + if (fileSystem.delete(appDir, true)) { + LOG.info("Successfully deleted service dir for " + serviceName + ": " + + appDir); + } else { + String message = + "Failed to delete service + " + serviceName + " at: " + appDir; + LOG.info(message); + throw new YarnException(message); + } + } + deleteZKNode(serviceName); + String registryPath = ServiceRegistryUtils.registryPathForInstance(serviceName); + try { + getRegistryClient().delete(registryPath, true); + } catch (IOException e) { + LOG.warn("Error deleting registry entry {}", registryPath, e); + } + LOG.info("Destroyed cluster {}", serviceName); + return EXIT_SUCCESS; + } + + private synchronized RegistryOperations getRegistryClient() + throws SliderException, IOException { + + if (registryClient == null) { + registryClient = + RegistryOperationsFactory.createInstance("ServiceClient", getConfig()); + registryClient.init(getConfig()); + registryClient.start(); + } + return registryClient; + } + + private void deleteZKNode(String clusterName) throws Exception { + CuratorFramework curatorFramework = getCuratorClient(); + String user = RegistryUtils.currentUser(); + String zkPath = ServiceRegistryUtils.mkClusterPath(user, clusterName); + if (curatorFramework.checkExists().forPath(zkPath) != null) { + curatorFramework.delete().deletingChildrenIfNeeded().forPath(zkPath); + LOG.info("Deleted zookeeper path: " + zkPath); + } + } + + private synchronized CuratorFramework getCuratorClient() + throws BadConfigException { + String registryQuorum = + getConfig().get(RegistryConstants.KEY_REGISTRY_ZK_QUORUM); + + // though if neither is set: trouble + if (SliderUtils.isUnset(registryQuorum)) { + throw new BadConfigException( + "No Zookeeper quorum provided in the" + " configuration property " + + RegistryConstants.KEY_REGISTRY_ZK_QUORUM); + } + ZookeeperUtils.splitToHostsAndPortsStrictly(registryQuorum); + + if (curatorClient == null) { + curatorClient = + CuratorFrameworkFactory.builder().connectString(registryQuorum) + .sessionTimeoutMs(10000).retryPolicy(new RetryNTimes(5, 2000)) + .build(); + curatorClient.start(); + } + return curatorClient; + } + + private int actionHelp(String actionName, CommonArgs args) + throws YarnException, IOException { + throw new UsageException(CommonArgs.usage(args, actionName)); + } + + private void verifyNoLiveAppInRM(String serviceName, String action) + throws IOException, YarnException { + Set types = new HashSet<>(1); + types.add(YarnServiceConstants.APP_TYPE); + Set tags = null; + if (serviceName != null) { + tags = Collections.singleton(SliderUtils.createNameTag(serviceName)); + } + GetApplicationsRequest request = GetApplicationsRequest.newInstance(); + request.setApplicationTypes(types); + request.setApplicationTags(tags); + request.setApplicationStates(liveStates); + List reports = yarnClient.getApplications(request); + if (!reports.isEmpty()) { + String message = ""; + if (action.equals("destroy")) { + message = "Failed to destroy service " + serviceName + + ", because it is still running."; + } else { + message = "Failed to " + action + " service " + serviceName + + ", because it already exists."; + } + throw new YarnException(message); + } + } + + private ApplicationId submitApp(Service app) + throws IOException, YarnException { + String serviceName = app.getName(); + Configuration conf = getConfig(); + Path appRootDir = fs.buildClusterDirPath(app.getName()); + + YarnClientApplication yarnApp = yarnClient.createApplication(); + ApplicationSubmissionContext submissionContext = + yarnApp.getApplicationSubmissionContext(); + ServiceApiUtil.validateCompResourceSize( + yarnApp.getNewApplicationResponse().getMaximumResourceCapability(), + app); + + submissionContext.setKeepContainersAcrossApplicationAttempts(true); + if (app.getLifetime() > 0) { + Map appTimeout = new HashMap<>(); + appTimeout.put(ApplicationTimeoutType.LIFETIME, app.getLifetime()); + submissionContext.setApplicationTimeouts(appTimeout); + } + submissionContext.setMaxAppAttempts(conf.getInt( + YarnServiceConf.AM_RESTART_MAX, 2)); + + Map localResources = new HashMap<>(); + + // copy local slideram-log4j.properties to hdfs and add to localResources + boolean hasAMLog4j = + addAMLog4jResource(serviceName, conf, localResources); + // copy jars to hdfs and add to localResources + addJarResource(serviceName, localResources); + // add keytab if in secure env + addKeytabResourceIfSecure(fs, localResources, conf, serviceName); + if (LOG.isDebugEnabled()) { + printLocalResources(localResources); + } + Map env = addAMEnv(conf); + + // create AM CLI + String cmdStr = + buildCommandLine(serviceName, conf, appRootDir, hasAMLog4j); + submissionContext.setResource(Resource.newInstance(YarnServiceConf + .getLong(YarnServiceConf.AM_RESOURCE_MEM, YarnServiceConf.DEFAULT_KEY_AM_RESOURCE_MEM, + app.getConfiguration(), conf), 1)); + String queue = app.getQueue(); + if (StringUtils.isEmpty(queue)) { + queue = conf.get(YARN_QUEUE, "default"); + } + submissionContext.setQueue(queue); + submissionContext.setApplicationName(serviceName); + submissionContext.setApplicationType(YarnServiceConstants.APP_TYPE); + Set appTags = + AbstractClientProvider.createApplicationTags(serviceName, null, null); + if (!appTags.isEmpty()) { + submissionContext.setApplicationTags(appTags); + } + ContainerLaunchContext amLaunchContext = + Records.newRecord(ContainerLaunchContext.class); + amLaunchContext.setCommands(Collections.singletonList(cmdStr)); + amLaunchContext.setEnvironment(env); + amLaunchContext.setLocalResources(localResources); + submissionContext.setAMContainerSpec(amLaunchContext); + yarnClient.submitApplication(submissionContext); + return submissionContext.getApplicationId(); + } + + private void printLocalResources(Map map) { + LOG.debug("Added LocalResource for localization: "); + StringBuilder builder = new StringBuilder(); + for (Map.Entry entry : map.entrySet()) { + builder.append(entry.getKey()).append(" -> ") + .append(entry.getValue().getResource().getFile()) + .append(System.lineSeparator()); + } + LOG.debug(builder.toString()); + } + + private String buildCommandLine(String serviceName, Configuration conf, + Path appRootDir, boolean hasSliderAMLog4j) throws BadConfigException { + JavaCommandLineBuilder CLI = new JavaCommandLineBuilder(); + CLI.forceIPv4().headless(); + //TODO CLI.setJVMHeap + //TODO CLI.addJVMOPTS + if (hasSliderAMLog4j) { + CLI.sysprop(SYSPROP_LOG4J_CONFIGURATION, YARN_SERVICE_LOG4J_FILENAME); + CLI.sysprop(SYSPROP_LOG_DIR, ApplicationConstants.LOG_DIR_EXPANSION_VAR); + } + CLI.add(ServiceMaster.class.getCanonicalName()); + CLI.add(ACTION_CREATE, serviceName); + //TODO debugAM CLI.add(Arguments.ARG_DEBUG) + CLI.add(Arguments.ARG_SERVICE_DEF_PATH, new Path(appRootDir, serviceName + ".json")); + // pass the registry binding + CLI.addConfOptionToCLI(conf, RegistryConstants.KEY_REGISTRY_ZK_ROOT, + RegistryConstants.DEFAULT_ZK_REGISTRY_ROOT); + CLI.addMandatoryConfOption(conf, RegistryConstants.KEY_REGISTRY_ZK_QUORUM); + + // write out the path output + CLI.addOutAndErrFiles(STDOUT_AM, STDERR_AM); + String cmdStr = CLI.build(); + LOG.info("AM launch command: {}", cmdStr); + return cmdStr; + } + + private Map addAMEnv(Configuration conf) throws IOException { + Map env = new HashMap<>(); + ClasspathConstructor classpath = + buildClasspath(YarnServiceConstants.SUBMITTED_CONF_DIR, "lib", fs, getConfig() + .getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, false)); + env.put("CLASSPATH", classpath.buildClasspath()); + env.put("LANG", "en_US.UTF-8"); + env.put("LC_ALL", "en_US.UTF-8"); + env.put("LANGUAGE", "en_US.UTF-8"); + String jaas = System.getenv("HADOOP_JAAS_DEBUG"); + if (jaas != null) { + env.put("HADOOP_JAAS_DEBUG", jaas); + } + if (!UserGroupInformation.isSecurityEnabled()) { + String userName = UserGroupInformation.getCurrentUser().getUserName(); + LOG.info("Run as user " + userName); + // HADOOP_USER_NAME env is used by UserGroupInformation when log in + // This env makes AM run as this user + env.put("HADOOP_USER_NAME", userName); + } + LOG.info("AM env: \n{}", stringifyMap(env)); + return env; + } + + protected Path addJarResource(String serviceName, + Map localResources) + throws IOException, SliderException { + Path libPath = fs.buildClusterDirPath(serviceName); + ProviderUtils + .addProviderJar(localResources, ServiceMaster.class, SERVICE_CORE_JAR, fs, + libPath, "lib", false); + Path dependencyLibTarGzip = fs.getDependencyTarGzip(); + if (fs.isFile(dependencyLibTarGzip)) { + LOG.info("Loading lib tar from " + fs.getFileSystem().getScheme() + ":/" + + dependencyLibTarGzip); + SliderUtils.putAmTarGzipAndUpdate(localResources, fs); + } else { + String[] libs = SliderUtils.getLibDirs(); + for (String libDirProp : libs) { + ProviderUtils.addAllDependencyJars(localResources, fs, libPath, "lib", + libDirProp); + } + } + return libPath; + } + + private boolean addAMLog4jResource(String serviceName, Configuration conf, + Map localResources) + throws IOException, BadClusterStateException { + boolean hasAMLog4j = false; + String hadoopConfDir = + System.getenv(ApplicationConstants.Environment.HADOOP_CONF_DIR.name()); + if (hadoopConfDir != null) { + File localFile = + new File(hadoopConfDir, YarnServiceConstants.YARN_SERVICE_LOG4J_FILENAME); + if (localFile.exists()) { + Path localFilePath = createLocalPath(localFile); + Path appDirPath = fs.buildClusterDirPath(serviceName); + Path remoteConfPath = + new Path(appDirPath, YarnServiceConstants.SUBMITTED_CONF_DIR); + Path remoteFilePath = + new Path(remoteConfPath, YarnServiceConstants.YARN_SERVICE_LOG4J_FILENAME); + copy(conf, localFilePath, remoteFilePath); + LocalResource localResource = + fs.createAmResource(remoteConfPath, LocalResourceType.FILE); + localResources.put(localFilePath.getName(), localResource); + hasAMLog4j = true; + } else { + LOG.warn("AM log4j property file doesn't exist: " + localFile); + } + } + return hasAMLog4j; + } + + public int actionStart(String serviceName) throws YarnException, IOException { + ServiceApiUtil.validateNameFormat(serviceName, getConfig()); + Path appDir = checkAppExistOnHdfs(serviceName); + Service service = ServiceApiUtil.loadService(fs, serviceName); + ServiceApiUtil.validateAndResolveService(service, fs, getConfig()); + // see if it is actually running and bail out; + verifyNoLiveAppInRM(serviceName, "thaw"); + ApplicationId appId = submitApp(service); + service.setId(appId.toString()); + // write app definition on to hdfs + createDirAndPersistApp(appDir, service); + return 0; + } + + private Path checkAppNotExistOnHdfs(Service service) + throws IOException, SliderException { + Path appDir = fs.buildClusterDirPath(service.getName()); + fs.verifyDirectoryNonexistent( + new Path(appDir, service.getName() + ".json")); + return appDir; + } + + private Path checkAppExistOnHdfs(String serviceName) + throws IOException, SliderException { + Path appDir = fs.buildClusterDirPath(serviceName); + fs.verifyPathExists(new Path(appDir, serviceName + ".json")); + return appDir; + } + + private void createDirAndPersistApp(Path appDir, Service service) + throws IOException, SliderException { + FsPermission appDirPermission = new FsPermission("750"); + fs.createWithPermissions(appDir, appDirPermission); + persistAppDef(appDir, service); + } + + private void persistAppDef(Path appDir, Service service) + throws IOException { + Path appJson = new Path(appDir, service.getName() + ".json"); + jsonSerDeser + .save(fs.getFileSystem(), appJson, service, true); + LOG.info( + "Persisted service " + service.getName() + " at " + appJson); + } + + private void addKeytabResourceIfSecure(SliderFileSystem fileSystem, + Map localResource, Configuration conf, + String serviceName) throws IOException, BadConfigException { + if (!UserGroupInformation.isSecurityEnabled()) { + return; + } + String keytabPreInstalledOnHost = + conf.get(YarnServiceConf.KEY_AM_KEYTAB_LOCAL_PATH); + if (StringUtils.isEmpty(keytabPreInstalledOnHost)) { + String amKeytabName = + conf.get(YarnServiceConf.KEY_AM_LOGIN_KEYTAB_NAME); + String keytabDir = conf.get(YarnServiceConf.KEY_HDFS_KEYTAB_DIR); + Path keytabPath = + fileSystem.buildKeytabPath(keytabDir, amKeytabName, serviceName); + if (fileSystem.getFileSystem().exists(keytabPath)) { + LocalResource keytabRes = + fileSystem.createAmResource(keytabPath, LocalResourceType.FILE); + localResource + .put(YarnServiceConstants.KEYTAB_DIR + "/" + amKeytabName, keytabRes); + LOG.info("Adding AM keytab on hdfs: " + keytabPath); + } else { + LOG.warn("No keytab file was found at {}.", keytabPath); + if (conf.getBoolean(YarnServiceConf.KEY_AM_LOGIN_KEYTAB_REQUIRED, false)) { + throw new BadConfigException("No keytab file was found at %s.", + keytabPath); + } else { + LOG.warn("The AM will be " + + "started without a kerberos authenticated identity. " + + "The service is therefore not guaranteed to remain " + + "operational beyond 24 hours."); + } + } + } + } + + public String updateLifetime(String serviceName, long lifetime) + throws YarnException, IOException { + ApplicationId currentAppId = getAppId(serviceName); + ApplicationReport report = yarnClient.getApplicationReport(currentAppId); + if (report == null) { + throw new YarnException("Service not found for " + serviceName); + } + ApplicationId appId = report.getApplicationId(); + LOG.info("Updating lifetime of an service: serviceName = " + serviceName + + ", appId = " + appId + ", lifetime = " + lifetime); + Map map = new HashMap<>(); + String newTimeout = + Times.formatISO8601(System.currentTimeMillis() + lifetime * 1000); + map.put(ApplicationTimeoutType.LIFETIME, newTimeout); + UpdateApplicationTimeoutsRequest request = + UpdateApplicationTimeoutsRequest.newInstance(appId, map); + yarnClient.updateApplicationTimeouts(request); + LOG.info( + "Successfully updated lifetime for an service: serviceName = " + serviceName + + ", appId = " + appId + ". New expiry time in ISO8601 format is " + + newTimeout); + return newTimeout; + } + + public ServiceState convertState(FinalApplicationStatus status) { + switch (status) { + case UNDEFINED: + return ServiceState.ACCEPTED; + case FAILED: + case KILLED: + return ServiceState.FAILED; + case ENDED: + case SUCCEEDED: + return ServiceState.STOPPED; + } + return ServiceState.ACCEPTED; + } + + public Service getStatus(String serviceName) + throws IOException, YarnException { + ServiceApiUtil.validateNameFormat(serviceName, getConfig()); + ApplicationId currentAppId = getAppId(serviceName); + ApplicationReport appReport = yarnClient.getApplicationReport(currentAppId); + Service appSpec = new Service(); + appSpec.setName(serviceName); + appSpec.setState(convertState(appReport.getFinalApplicationStatus())); + ApplicationTimeout lifetime = + appReport.getApplicationTimeouts().get(ApplicationTimeoutType.LIFETIME); + if (lifetime != null) { + appSpec.setLifetime(lifetime.getRemainingTime()); + } + + if (appReport.getYarnApplicationState() != RUNNING) { + LOG.info("Service {} is at {} state", serviceName, + appReport.getYarnApplicationState()); + return appSpec; + } + if (StringUtils.isEmpty(appReport.getHost())) { + LOG.warn(serviceName + " AM hostname is empty"); + return appSpec; + } + ClientAMProtocol amProxy = + createAMProxy(appReport.getHost(), appReport.getRpcPort()); + GetStatusResponseProto response = + amProxy.getStatus(GetStatusRequestProto.newBuilder().build()); + appSpec = jsonSerDeser.fromJson(response.getStatus()); + + return appSpec; + } + + public YarnClient getYarnClient() { + return this.yarnClient; + } + + public int actionDependency(ActionDependencyArgs args) + throws IOException, YarnException { + String currentUser = RegistryUtils.currentUser(); + LOG.info("Running command as user {}", currentUser); + + Path dependencyLibTarGzip = fs.getDependencyTarGzip(); + + // Check if dependency has already been uploaded, in which case log + // appropriately and exit success (unless overwrite has been requested) + if (fs.isFile(dependencyLibTarGzip) && !args.overwrite) { + System.out.println(String.format( + "Dependency libs are already uploaded to %s. Use %s " + + "if you want to re-upload", dependencyLibTarGzip.toUri(), + Arguments.ARG_OVERWRITE)); + return EXIT_SUCCESS; + } + + String[] libDirs = SliderUtils.getLibDirs(); + if (libDirs.length > 0) { + File tempLibTarGzipFile = File.createTempFile( + YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_NAME + "_", + YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_EXT); + // copy all jars + tarGzipFolder(libDirs, tempLibTarGzipFile, createJarFilter()); + + LOG.info("Version Info: " + VersionInfo.getBuildVersion()); + fs.copyLocalFileToHdfs(tempLibTarGzipFile, dependencyLibTarGzip, + new FsPermission(YarnServiceConstants.DEPENDENCY_DIR_PERMISSIONS)); + return EXIT_SUCCESS; + } else { + return EXIT_FALSE; + } + } + + protected ClientAMProtocol createAMProxy(String host, int port) + throws IOException { + InetSocketAddress address = + NetUtils.createSocketAddrForHost(host, port); + return ClientAMProxy.createProxy(getConfig(), ClientAMProtocol.class, + UserGroupInformation.getCurrentUser(), rpc, address); + } + + private synchronized ApplicationId getAppId(String serviceName) + throws IOException, YarnException { + if (cachedAppIds.containsKey(serviceName)) { + return cachedAppIds.get(serviceName); + } + Service persistedService = ServiceApiUtil.loadService(fs, serviceName); + if (persistedService == null) { + throw new YarnException("Service " + serviceName + + " doesn't exist on hdfs. Please check if the app exists in RM"); + } + ApplicationId currentAppId = ApplicationId.fromString(persistedService.getId()); + cachedAppIds.put(serviceName, currentAppId); + return currentAppId; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractActionArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractActionArgs.java new file mode 100644 index 0000000..9b7e2a4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractActionArgs.java @@ -0,0 +1,139 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.ErrorStrings; +import org.apache.hadoop.yarn.service.exceptions.UsageException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +/** + * Base args for all actions + */ +public abstract class AbstractActionArgs extends ArgOps implements Arguments { + protected static final Logger log = + LoggerFactory.getLogger(AbstractActionArgs.class); + + + protected AbstractActionArgs() { + } + + /** + * This is the default parameter + */ + @Parameter + public List parameters = new ArrayList<>(); + + /** + * get the name: relies on arg 1 being the cluster name in all operations + * @return the name argument, null if there is none + */ + public String getServiceName() { + return (parameters.isEmpty()) ? null : parameters.get(0); + } + + /** + -D name=value + + Define an configuration option which overrides any options in + the configuration XML files of the image or in the image configuration + directory. The values will be persisted. + Configuration options are only passed to the cluster when creating or reconfiguring a cluster. + + */ + + @Parameter(names = ARG_DEFINE, arity = 1, description = "Definitions", hidden = true) + public List definitions = new ArrayList<>(); + + /** + * System properties + */ + @Parameter(names = {ARG_SYSPROP}, arity = 1, + description = "system properties in the form name value" + + " These are set after the JVM is started.", + hidden = true) + public List sysprops = new ArrayList<>(0); + + + @Parameter(names = ARG_DEBUG, description = "Debug mode", hidden = true) + public boolean debug = false; + + /** + * Get the min #of params expected + * @return the min number of params in the {@link #parameters} field + */ + public int getMinParams() { + return 1; + } + + /** + * Get the name of the action + * @return the action name + */ + public abstract String getActionName() ; + + /** + * Get the max #of params expected + * @return the number of params in the {@link #parameters} field; + */ + public int getMaxParams() { + return getMinParams(); + } + + public void validate() throws BadCommandArgumentsException, UsageException { + + int minArgs = getMinParams(); + int actionArgSize = parameters.size(); + if (minArgs > actionArgSize) { + throw new BadCommandArgumentsException( + ErrorStrings.ERROR_NOT_ENOUGH_ARGUMENTS + getActionName() + + ", Expected minimum " + minArgs + " but got " + actionArgSize); + } + int maxArgs = getMaxParams(); + if (maxArgs == -1) { + maxArgs = minArgs; + } + if (actionArgSize > maxArgs) { + String message = String.format("%s for action %s: limit is %d but saw %d: ", + ErrorStrings.ERROR_TOO_MANY_ARGUMENTS, + getActionName(), maxArgs, + actionArgSize); + + log.error(message); + int index = 1; + StringBuilder buf = new StringBuilder(message); + for (String actionArg : parameters) { + log.error("[{}] \"{}\"", index++, actionArg); + buf.append(" \"").append(actionArg).append("\" "); + } + throw new BadCommandArgumentsException(buf.toString()); + } + } + + @Override + public String toString() { + return super.toString() + ": " + getActionName(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractArgsDelegate.java new file mode 100644 index 0000000..457e357 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractArgsDelegate.java @@ -0,0 +1,28 @@ +/* + * 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.service.client.params; + +import org.apache.hadoop.yarn.service.client.params.ArgOps; +import org.apache.hadoop.yarn.service.client.params.Arguments; + +/** + * Base class for all the delegates + */ +public class AbstractArgsDelegate extends ArgOps implements Arguments { +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractClusterBuildingActionArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractClusterBuildingActionArgs.java new file mode 100644 index 0000000..4ecbe9c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractClusterBuildingActionArgs.java @@ -0,0 +1,46 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; + +import java.io.File; + +/** + * Abstract Action to build things; shares args across build and + * list + */ +public abstract class AbstractClusterBuildingActionArgs + extends AbstractActionArgs { + @Parameter(names = { ARG_FILE, ARG_FILE_SHORT }, + description = "The path to the service definition file in JSON format.") + public File file; + + public File getFile() { + return file; + } + + @Parameter(names = { + ARG_QUEUE, ARG_SHORT_QUEUE}, description = "Queue to submit the service") + public String queue; + + @Parameter(names = { + ARG_LIFETIME }, description = "Lifetime of the service from the time of request") + public long lifetime; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionBuildArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionBuildArgs.java new file mode 100644 index 0000000..0228125 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionBuildArgs.java @@ -0,0 +1,39 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameters; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; + +@Parameters(commandNames = { SliderActions.ACTION_BUILD}, + commandDescription = SliderActions.DESCRIBE_ACTION_BUILD) + +public class ActionBuildArgs extends AbstractClusterBuildingActionArgs { + + @Override + public String getActionName() { + return SliderActions.ACTION_BUILD; + } + + @Override + public void validate() throws BadCommandArgumentsException { + if (file == null) { + throw new BadCommandArgumentsException("No service definition provided."); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionClientArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionClientArgs.java new file mode 100644 index 0000000..c43d61a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionClientArgs.java @@ -0,0 +1,71 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; +import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; +import org.apache.hadoop.yarn.service.client.params.SliderActions; + +import java.io.File; + +@Parameters(commandNames = { SliderActions.ACTION_CLIENT}, + commandDescription = SliderActions.DESCRIBE_ACTION_CLIENT) + +public class ActionClientArgs extends AbstractActionArgs { + + @Override + public String getActionName() { + return SliderActions.ACTION_CLIENT; + } + + @Parameter(names = {ARG_INSTALL}, + description = "Install client") + public boolean install; + + @Parameter(names = {ARG_NAME}, + description = "The name of the service") + public String name; + + @Parameter(names = {ARG_PACKAGE}, + description = "Path to app package") + public String packageURI; + + @Parameter(names = {ARG_DEST}, + description = "The location where to install the client") + public File installLocation; + + @Parameter(names = {ARG_CONFIG}, + description = "Client configuration") + public File clientConfig; + + /** + * Get the min #of params expected + * + * @return the min number of params in the {@link #parameters} field + */ + public int getMinParams() { + return 0; + } + + @Override + public int getMaxParams() { + return 1; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionCreateArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionCreateArgs.java new file mode 100644 index 0000000..039f528 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionCreateArgs.java @@ -0,0 +1,46 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; + +@Parameters(commandNames = { SliderActions.ACTION_CREATE}, + commandDescription = SliderActions.DESCRIBE_ACTION_CREATE) + +public class ActionCreateArgs extends AbstractClusterBuildingActionArgs { + + @Parameter(names = { ARG_EXAMPLE, ARG_EXAMPLE_SHORT }, + description = "The name of the example service such as sleeper") + public String example; + + @Override + public String getActionName() { + return SliderActions.ACTION_CREATE; + } + + @Override + public void validate() throws BadCommandArgumentsException { + if (file == null && example == null) { + throw new BadCommandArgumentsException("No service definition provided."); + } + } +} + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionDependencyArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionDependencyArgs.java new file mode 100644 index 0000000..699e6ab --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionDependencyArgs.java @@ -0,0 +1,65 @@ +/* + * 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.service.client.params; + +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.UsageException; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; + +@Parameters(commandNames = { SliderActions.ACTION_DEPENDENCY }, + commandDescription = SliderActions.DESCRIBE_ACTION_DEPENDENCY) +public class ActionDependencyArgs extends AbstractActionArgs { + + @Override + public String getActionName() { + return SliderActions.ACTION_DEPENDENCY; + } + + @Parameter(names = { ARG_UPLOAD }, required = true, + description = "Upload AM libraries to HDFS for this client version") + public boolean upload; + + @Parameter(names = { ARG_OVERWRITE }, + description = "Overwrite current uploaded dependency libs") + public boolean overwrite = false; + + /** + * Get the min #of params expected + * + * @return the min number of params in the {@link #parameters} field + */ + public int getMinParams() { + return 0; + } + + @Override + public int getMaxParams() { + return 1; + } + + @Override + public void validate() throws BadCommandArgumentsException, UsageException { + super.validate(); + + if (!upload) { + throw new UsageException("Option " + ARG_UPLOAD + " is mandatory"); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionDestroyArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionDestroyArgs.java new file mode 100644 index 0000000..8c41c04 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionDestroyArgs.java @@ -0,0 +1,37 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; + +@Parameters(commandNames = { SliderActions.ACTION_DESTROY}, + commandDescription = SliderActions.DESCRIBE_ACTION_DESTROY) + +public class ActionDestroyArgs extends AbstractActionArgs { + + @Override + public String getActionName() { + return SliderActions.ACTION_DESTROY; + } + + @Parameter(names = {ARG_FORCE}, + description = "force the operation") + public boolean force; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionExistsArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionExistsArgs.java new file mode 100644 index 0000000..088ad47 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionExistsArgs.java @@ -0,0 +1,49 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; +import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; +import org.apache.hadoop.yarn.service.client.params.SliderActions; + +import java.io.File; + +@Parameters(commandNames = { SliderActions.ACTION_EXISTS}, + commandDescription = SliderActions.DESCRIBE_ACTION_EXISTS) + +public class ActionExistsArgs extends AbstractActionArgs { + + @Override + public String getActionName() { + return SliderActions.ACTION_EXISTS; + } + + @Parameter(names = {ARG_LIVE}, + description = "verify that the service is running") + public boolean live; + + @Parameter(names = {ARG_STATE}, + description = "verify that the service is in the specific YARN state") + public String state = ""; + + @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT}, + description = "output file for any service report") + public File out; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionFlexArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionFlexArgs.java new file mode 100644 index 0000000..b7acf58 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionFlexArgs.java @@ -0,0 +1,50 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameters; +import com.beust.jcommander.ParametersDelegate; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; + +import java.util.List; +import java.util.Map; + +@Parameters(commandNames = { SliderActions.ACTION_FLEX}, + commandDescription = SliderActions.DESCRIBE_ACTION_FLEX) + +public class ActionFlexArgs extends AbstractActionArgs { + + @Override + public String getActionName() { + return SliderActions.ACTION_FLEX; + } + + @ParametersDelegate + public ComponentArgsDelegate componentDelegate = new ComponentArgsDelegate(); + + /** + * Get the component mapping (may be empty, but never null) + * @return mapping + * @throws BadCommandArgumentsException parse problem + */ + public Map getComponentMap() throws + BadCommandArgumentsException { + return componentDelegate.getComponentMap(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionFreezeArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionFreezeArgs.java new file mode 100644 index 0000000..aecf0eb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionFreezeArgs.java @@ -0,0 +1,56 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; +import com.beust.jcommander.ParametersDelegate; + +@Parameters(commandNames = { SliderActions.ACTION_STOP }, + commandDescription = SliderActions.DESCRIBE_ACTION_FREEZE) + +public class ActionFreezeArgs extends AbstractActionArgs implements + WaitTimeAccessor { + @Override + public String getActionName() { + return SliderActions.ACTION_STOP; + } + + public static final String FREEZE_COMMAND_ISSUED = "stop command issued"; + @ParametersDelegate + public WaitArgsDelegate waitDelegate = new WaitArgsDelegate(); + + @Override + public int getWaittime() { + return waitDelegate.getWaittime(); + } + + @Override + public void setWaittime(int waittime) { + waitDelegate.setWaittime(waittime); + } + + @Parameter(names={ARG_MESSAGE}, + description = "reason for the operation") + public String message = FREEZE_COMMAND_ISSUED; + + @Parameter(names = {ARG_FORCE}, + description = "force the operation") + public boolean force; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionHelpArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionHelpArgs.java new file mode 100644 index 0000000..51aa88a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionHelpArgs.java @@ -0,0 +1,44 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameters; +import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; +import org.apache.hadoop.yarn.service.client.params.SliderActions; + +/** + * The Help command + */ +@Parameters(commandNames = { SliderActions.ACTION_HELP}, + commandDescription = SliderActions.DESCRIBE_ACTION_HELP) +public class ActionHelpArgs extends AbstractActionArgs { + @Override + public String getActionName() { + return SliderActions.ACTION_HELP; + } + + /** + * Get the min #of params expected + * @return the min number of params in the {@link #parameters} field + */ + @Override + public int getMinParams() { + return 0; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionKeytabArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionKeytabArgs.java new file mode 100644 index 0000000..7e51457 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionKeytabArgs.java @@ -0,0 +1,76 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; +import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; +import org.apache.hadoop.yarn.service.client.params.SliderActions; + +@Parameters(commandNames = { SliderActions.ACTION_KEYTAB}, + commandDescription = SliderActions.DESCRIBE_ACTION_KEYTAB) + +public class ActionKeytabArgs extends AbstractActionArgs { + + public ActionKeytabArgs() { + super(); + } + + @Override + public String getActionName() { + return SliderActions.ACTION_INSTALL_KEYTAB; + } + + @Parameter(names = {ARG_KEYTABINSTALL}, + description = "Install the keytab") + public boolean install; + + @Parameter(names = {ARG_KEYTABDELETE}, + description = "Delete the keytab") + public boolean delete; + + @Parameter(names = {ARG_KEYTABLIST}, + description = "List of installed keytabs") + public boolean list; + + @Parameter(names = {ARG_KEYTAB}, + description = "Path or name of the keytab") + public String keytab; + + @Parameter(names = {ARG_FOLDER}, + description = "The name of the folder in which to store the keytab") + public String folder; + + @Parameter(names = {ARG_OVERWRITE}, description = "Overwrite existing keytab") + public boolean overwrite = false; + + /** + * Get the min #of params expected + * @return the min number of params in the {@link #parameters} field + */ + public int getMinParams() { + return 0; + } + + @Override + public int getMaxParams() { + return 3; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionListArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionListArgs.java new file mode 100644 index 0000000..c05e602 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionListArgs.java @@ -0,0 +1,76 @@ +/* + * 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.service.client.params; + +import java.util.HashSet; +import java.util.Set; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; +import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; +import org.apache.hadoop.yarn.service.client.params.SliderActions; + +@Parameters(commandNames = { SliderActions.ACTION_LIST}, + commandDescription = SliderActions.DESCRIBE_ACTION_LIST) + +public class ActionListArgs extends AbstractActionArgs { + @Override + public String getActionName() { + return SliderActions.ACTION_LIST; + } + + @Parameter(names = {ARG_LIVE}, + description = "List only live service instances") + public boolean live; + + @Parameter(names = {ARG_STATE}, + description = "list only applications in the specific YARN state") + public String state = ""; + + @Parameter(names = {ARG_VERBOSE}, + description = "print out information in details") + public boolean verbose = false; + + @Parameter(names = {ARG_CONTAINERS}, + description = "List containers of a service instance") + public boolean containers; + + @Parameter(names = {ARG_VERSION}, + description = "Filter containers by app version (used with " + + ARG_CONTAINERS + ")") + public String version; + + @Parameter(names = {ARG_COMPONENTS}, variableArity = true, + description = "Filter containers by component names (used with " + + ARG_CONTAINERS + ")") + public Set components = new HashSet<>(0); + + /** + * Get the min #of params expected + * @return the min number of params in the {@link #parameters} field + */ + public int getMinParams() { + return 0; + } + + @Override + public int getMaxParams() { + return 1; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionRegistryArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionRegistryArgs.java new file mode 100644 index 0000000..3e53418 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionRegistryArgs.java @@ -0,0 +1,218 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; +import org.apache.hadoop.yarn.service.conf.YarnServiceConstants; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.UsageException; +import org.apache.hadoop.yarn.service.api.records.ConfigFormat; + +import static org.apache.hadoop.yarn.service.client.params.SliderActions.ACTION_REGISTRY; +import static org.apache.hadoop.yarn.service.client.params.SliderActions.DESCRIBE_ACTION_REGISTRY; +import java.io.File; + +/** + * Registry actions + * + * --instance {app name}, if a / is in it, refers underneath? + * --dest {destfile} + * --list : list instances of slider service + * --listfiles + */ +@Parameters(commandNames = {ACTION_REGISTRY}, + commandDescription = DESCRIBE_ACTION_REGISTRY) + +public class ActionRegistryArgs extends AbstractActionArgs { + + public static final String USAGE = + "Usage: " + SliderActions.ACTION_REGISTRY + + " (" + + Arguments.ARG_LIST + "|" + + Arguments.ARG_LISTCONF + "|" + + Arguments.ARG_LISTEXP + "|" + + Arguments.ARG_LISTFILES + "|" + + Arguments.ARG_GETCONF + "|" + + Arguments.ARG_GETEXP + "> " + + Arguments.ARG_NAME + " " + + " )" + + "[" + Arguments.ARG_VERBOSE + "] " + + "[" + Arguments.ARG_USER + "] " + + "[" + Arguments.ARG_OUTPUT + " ] " + + "[" + Arguments.ARG_SERVICETYPE + " ] " + + "[" + Arguments.ARG_FORMAT + " ] " + + System.getProperty("line.separator") + + "Arguments.ARG_GETEXP only supports " + Arguments.ARG_FORMAT + " json" + ; + public ActionRegistryArgs() { + } + + public ActionRegistryArgs(String name) { + this.name = name; + } + + @Override + public String getActionName() { + return ACTION_REGISTRY; + } + + /** + * Get the min #of params expected + * @return the min number of params in the {@link #parameters} field + */ + @Override + public int getMinParams() { + return 0; + } + + @Parameter(names = {ARG_LIST}, + description = "list services") + public boolean list; + + @Parameter(names = {ARG_LISTCONF}, + description = "list configurations") + public boolean listConf; + + @Parameter(names = {ARG_GETCONF}, + description = "get configuration") + public String getConf; + + @Parameter(names = {ARG_LISTEXP}, + description = "list exports") + public boolean listExports; + + @Parameter(names = {ARG_GETEXP}, + description = "get export") + public String getExport; + + @Parameter(names = {ARG_LISTFILES}, + description = "list files") + public String listFiles; + + @Parameter(names = {ARG_GETFILES}, + description = "get files") + public String getFiles; + + //--format + @Parameter(names = ARG_FORMAT, + description = "Format for a response: ") + public String format = ConfigFormat.XML.toString() ; + + @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT, ARG_DEST}, + description = "Output destination") + public File out; + + @Parameter(names = {ARG_NAME}, + description = "name of an instance") + public String name; + + @Parameter(names = {ARG_SERVICETYPE}, + description = "optional service type") + public String serviceType = YarnServiceConstants.APP_TYPE; + + @Parameter(names = {ARG_VERBOSE}, + description = "verbose output") + public boolean verbose; + + @Parameter(names = {ARG_INTERNAL}, + description = "fetch internal registry entries") + public boolean internal; + + @Parameter(names = {ARG_USER}, + description = "the name of the user whose service is being resolved") + public String user; + + /** + * validate health of all the different operations + * @throws BadCommandArgumentsException + */ + @Override + public void validate() throws BadCommandArgumentsException, UsageException { + super.validate(); + + //verify that at most one of the operations is set + int gets = s(getConf) + s(getFiles) + s(getExport); + int lists = s(list) + s(listConf) + s(listFiles) + s(listExports); + int set = lists + gets; + if (set > 1) { + throw new UsageException(USAGE); + } + + if (out != null && ( set == 0)) { + throw new UsageException("output path" + + " is only supported on 'get' operations: "); + } + if (!list && !is(name)) { + throw new UsageException("Argument " + ARG_NAME + +" missing: "); + + } + } + + private int s(String arg) { + return is(arg) ? 1 : 0; + } + + private boolean is(String arg) { + return arg != null; + } + + private int s(boolean arg) { + return arg ? 1 : 0; + } + + private String ifdef(String arg, boolean val) { + return val ? (arg + " "): ""; + } + + private String ifdef(String arg, String val) { + if (is(val)) { + return arg + " " + val + " "; + } else { + return ""; + } + } + + @Override + public String toString() { + final StringBuilder sb = + new StringBuilder(ACTION_REGISTRY); + sb.append(' '); + sb.append(ifdef(ARG_LIST, list)); + sb.append(ifdef(ARG_LISTCONF, listConf)); + sb.append(ifdef(ARG_LISTFILES, listFiles)); + sb.append(ifdef(ARG_GETCONF, getConf)); + sb.append(ifdef(ARG_GETFILES, getFiles)); + + sb.append(ifdef(ARG_NAME, name)); + sb.append(ifdef(ARG_SERVICETYPE, serviceType)); + + + sb.append(ifdef(ARG_VERBOSE, verbose)); + sb.append(ifdef(ARG_INTERNAL, internal)); + + if (out != null) { + sb.append(ifdef(ARG_OUTPUT, out.toString())); + } + sb.append(ifdef(ARG_FORMAT, format)); + + return sb.toString(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionResolveArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionResolveArgs.java new file mode 100644 index 0000000..65f0472 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionResolveArgs.java @@ -0,0 +1,153 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.UsageException; + +import java.io.File; + +import static org.apache.hadoop.yarn.service.client.params.SliderActions.ACTION_RESOLVE; +import static org.apache.hadoop.yarn.service.client.params.SliderActions.DESCRIBE_ACTION_REGISTRY; + +/** + * Resolve registry entries + * + * --path {path} + * --out {destfile} + * --verbose + * --list + */ +@Parameters(commandNames = {ACTION_RESOLVE}, + commandDescription = DESCRIBE_ACTION_REGISTRY) +public class ActionResolveArgs extends AbstractActionArgs { + + public static final String USAGE = + "Usage: " + SliderActions.ACTION_RESOLVE + + " " + + ARG_PATH + " " + + "[" + ARG_LIST + "] " + + "[" + ARG_OUTPUT + " ] " + + "[" + ARG_DESTDIR + " ] " + ; + public ActionResolveArgs() { + } + + @Override + public String getActionName() { + return ACTION_RESOLVE; + } + + /** + * Get the min #of params expected + * @return the min number of params in the {@link #parameters} field + */ + @Override + public int getMinParams() { + return 0; + } + + @Parameter(names = {ARG_LIST}, + description = "list services") + public boolean list; + + @Parameter(names = {ARG_PATH}, + description = "resolve a path") + public String path; + + @Parameter(names = {ARG_DESTDIR}, + description = "destination directory for operations") + public File destdir; + + @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT}, + description = "dest file") + public File out; + + @Override + public String toString() { + final StringBuilder sb = + new StringBuilder(ACTION_RESOLVE).append(" "); + sb.append(ARG_PATH).append(" ").append(path).append(" "); + if (list) { + sb.append(ARG_LIST).append(" "); + } + if (destdir != null) { + sb.append(ARG_DESTDIR).append(" ").append(destdir).append(" "); + } + if (out != null) { + sb.append(ARG_OUTPUT).append(" ").append(out).append(" "); + } + return sb.toString(); + } + + @Override + public void validate() throws BadCommandArgumentsException, UsageException { + super.validate(); + if (StringUtils.isEmpty(path)) { + throw new BadCommandArgumentsException("Missing mandatory argument " + + ARG_PATH); + } + if (list && out != null) { + throw new BadCommandArgumentsException("Argument " + + ARG_OUTPUT + + " not supported for " + ARG_LIST); + } + if (out != null && destdir != null) { + throw new BadCommandArgumentsException( + ARG_OUTPUT + " and " + ARG_DESTDIR + " cannot be used together" + ); + } + } + + public String getPath() { + return path; + } + + public void setPath(String path) { + this.path = path; + } + + public boolean isList() { + return list; + } + + public void setList(boolean list) { + this.list = list; + } + + public File getDestdir() { + return destdir; + } + + public void setDestdir(File destdir) { + this.destdir = destdir; + } + + public File getOut() { + return out; + } + + public void setOut(File out) { + this.out = out; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionResourceArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionResourceArgs.java new file mode 100644 index 0000000..b03dc92 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionResourceArgs.java @@ -0,0 +1,70 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; +import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; +import org.apache.hadoop.yarn.service.client.params.SliderActions; + +@Parameters(commandNames = { SliderActions.ACTION_RESOURCE}, + commandDescription = SliderActions.DESCRIBE_ACTION_RESOURCE) + +public class ActionResourceArgs extends AbstractActionArgs { + + @Override + public String getActionName() { + return SliderActions.ACTION_RESOURCE; + } + + @Parameter(names = {ARG_INSTALL}, + description = "Install the resource(s)") + public boolean install; + + @Parameter(names = {ARG_DELETE}, + description = "Delete the file") + public boolean delete; + + @Parameter(names = {ARG_LIST}, + description = "List of installed files") + public boolean list; + + @Parameter(names = {ARG_RESOURCE}, + description = "Name of the file or directory") + public String resource; + + @Parameter(names = {ARG_DESTDIR}, + description = "The name of the folder in which to store the resources") + public String folder; + + @Parameter(names = {ARG_OVERWRITE}, description = "Overwrite existing resource(s)") + public boolean overwrite = false; + + /** + * Get the min #of params expected + * @return the min number of params in the {@link #parameters} field + */ + public int getMinParams() { + return 0; + } + + @Override + public int getMaxParams() { + return 3; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionStatusArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionStatusArgs.java new file mode 100644 index 0000000..31f25ef --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionStatusArgs.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.service.client.params; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; +import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; +import org.apache.hadoop.yarn.service.client.params.SliderActions; + +@Parameters(commandNames = { SliderActions.ACTION_STATUS}, + commandDescription = SliderActions.DESCRIBE_ACTION_STATUS) + +public class ActionStatusArgs extends AbstractActionArgs { + + @Override + public String getActionName() { + return SliderActions.ACTION_STATUS; + } + + @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT}, + description = "Output file for the status information") + public String output; + + @Parameter(names = {ARG_LIFETIME}, + description = "Lifetime of the service from the time of request") + public boolean lifetime; + + public String getOutput() { + return output; + } + + public void setOutput(String output) { + this.output = output; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionThawArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionThawArgs.java new file mode 100644 index 0000000..175e367 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionThawArgs.java @@ -0,0 +1,67 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; +import com.beust.jcommander.ParametersDelegate; + +import java.io.File; + +@Parameters(commandNames = { SliderActions.ACTION_START }, + commandDescription = SliderActions.DESCRIBE_ACTION_THAW) +public class ActionThawArgs extends AbstractActionArgs implements + WaitTimeAccessor, + LaunchArgsAccessor { + + + @Override + public String getActionName() { + return SliderActions.ACTION_START; + } + + @Override + public int getWaittime() { + return launchArgs.getWaittime(); + } + + @ParametersDelegate + LaunchArgsDelegate launchArgs = new LaunchArgsDelegate(); + + @Parameter(names = {ARG_LIFETIME}, + description = "Life time of the service since service started at" + + " running state") + public long lifetime; + + @Override + public String getRmAddress() { + return launchArgs.getRmAddress(); + } + + @Override + public void setWaittime(int waittime) { + launchArgs.setWaittime(waittime); + } + + + @Override + public File getOutputFile() { + return launchArgs.getOutputFile(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionTokensArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionTokensArgs.java new file mode 100644 index 0000000..cf48513 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionTokensArgs.java @@ -0,0 +1,78 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.UsageException; + +import java.io.File; + +@Parameters(commandNames = { SliderActions.ACTION_TOKENS}, + commandDescription = "save tokens to a file or list tokens in a file") +public class ActionTokensArgs extends AbstractActionArgs { + + public static final String DUPLICATE_ARGS = "Only one of " + + ARG_SOURCE + " and " + ARG_OUTPUT + " allowed"; + + public static final String MISSING_KT_PROVIDER = + "Both " + ARG_KEYTAB + " and " + ARG_PRINCIPAL + + " must be provided"; + + @Override + public String getActionName() { + return SliderActions.ACTION_TOKENS; + } + + @Parameter(names = {ARG_OUTPUT}, + description = "File to write") + public File output; + + @Parameter(names = {ARG_SOURCE}, + description = "source file") + public File source; + + @Parameter(names = {ARG_KEYTAB}, description = "keytab to use") + public File keytab; + + @Parameter(names = {ARG_PRINCIPAL}, description = "principal to log in from a keytab") + public String principal=""; + + /** + * Get the min #of params expected + * @return the min number of params in the {@link #parameters} field + */ + public int getMinParams() { + return 0; + } + + @Override + public void validate() throws BadCommandArgumentsException, UsageException { + super.validate(); + if (output != null && source != null) { + throw new BadCommandArgumentsException(DUPLICATE_ARGS); + } + + // this is actually a !xor + if (keytab != null ^ !principal.isEmpty()) { + throw new BadCommandArgumentsException(MISSING_KT_PROVIDER); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionUpdateArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionUpdateArgs.java new file mode 100644 index 0000000..00af69a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionUpdateArgs.java @@ -0,0 +1,37 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; + +@Parameters(commandNames = { SliderActions.ACTION_UPDATE}, + commandDescription = SliderActions.DESCRIBE_ACTION_UPDATE) + +public class ActionUpdateArgs extends AbstractActionArgs { + + @Parameter(names = { + ARG_LIFETIME }, description = "Lifetime of the service from the time of request") + public long lifetime; + + @Override + public String getActionName() { + return SliderActions.ACTION_UPDATE; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ArgOps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ArgOps.java new file mode 100644 index 0000000..00151f4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ArgOps.java @@ -0,0 +1,156 @@ +/* + * 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.service.client.params; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.ErrorStrings; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Static argument manipulation operations + */ +public class ArgOps { + + private static final Logger + log = LoggerFactory.getLogger(ArgOps.class); + + /** + * create a 3-tuple + */ + public static List triple(String msg, int min, int max) { + List l = new ArrayList<>(3); + l.add(msg); + l.add(min); + l.add(max); + return l; + } + + public static void applyFileSystemBinding(String filesystemBinding, + Configuration conf) { + if (filesystemBinding != null) { + //filesystem argument was set -this overwrites any defaults in the + //configuration + FileSystem.setDefaultUri(conf, filesystemBinding); + } + } + + public static void splitPairs(Collection pairs, + Map dest) { + for (String prop : pairs) { + String[] keyval = prop.split("=", 2); + if (keyval.length == 2) { + dest.put(keyval[0], keyval[1]); + } + } + } + + + public static void applyDefinitions(Map definitionMap, + Configuration conf) { + for (Map.Entry entry : definitionMap.entrySet()) { + String key = entry.getKey(); + String val = entry.getValue(); + log.debug("configuration[{}]<=\"{}\"", key, val); + conf.set(key, val, "command line"); + } + } + + /** + * Create a map from a tuple list like ['worker','2','master','1] into a map + * ['worker':'2',"master":'1']; + * Duplicate entries also trigger errors + * @param description description for errors + * @param list list to conver to tuples + * @return the map of key value pairs -unordered. + * @throws BadCommandArgumentsException odd #of arguments received + */ + public static Map convertTupleListToMap(String description, + List list) throws + BadCommandArgumentsException { + Map results = new HashMap<>(); + if (list != null && !list.isEmpty()) { + int size = list.size(); + if (size % 2 != 0) { + //odd number of elements, not permitted + throw new BadCommandArgumentsException( + ErrorStrings.ERROR_PARSE_FAILURE + description); + } + for (int count = 0; count < size; count += 2) { + String key = list.get(count); + String val = list.get(count + 1); + if (results.get(key) != null) { + throw new BadCommandArgumentsException( + ErrorStrings.ERROR_DUPLICATE_ENTRY + description + + ": " + key); + } + results.put(key, val); + } + } + return results; + } + + /** + * Create a map from a tuple list like + * ['worker','heapsize','5G','master','heapsize','2M'] into a map + * ['worker':'2',"master":'1']; + * Duplicate entries also trigger errors + + * @throws BadCommandArgumentsException odd #of arguments received + */ + public static Map> convertTripleListToMaps(String description, + List list) throws BadCommandArgumentsException { + + Map> results = new HashMap<>(); + if (list != null && !list.isEmpty()) { + int size = list.size(); + if (size % 3 != 0) { + //wrong number of elements, not permitted + throw new BadCommandArgumentsException( + ErrorStrings.ERROR_PARSE_FAILURE + description); + } + for (int count = 0; count < size; count += 3) { + String role = list.get(count); + String key = list.get(count + 1); + String val = list.get(count + 2); + Map roleMap = results.get(role); + if (roleMap == null) { + //demand create new role map + roleMap = new HashMap<>(); + results.put(role, roleMap); + } + if (roleMap.get(key) != null) { + throw new BadCommandArgumentsException( + ErrorStrings.ERROR_DUPLICATE_ENTRY + description + + ": for key " + key + " under " + role); + } + roleMap.put(key, val); + } + } + return results; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/Arguments.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/Arguments.java new file mode 100644 index 0000000..67571e2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/Arguments.java @@ -0,0 +1,103 @@ +/* + * 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.service.client.params; + +/** + * Here are all the arguments that may be parsed by the client or server + * command lines. + * + * Important: Please keep the main list in alphabetical order + * so it is easier to see what arguments are there + */ +public interface Arguments { + + String ARG_FILE = "--file"; + String ARG_FILE_SHORT = "-f"; + String ARG_BASE_PATH = "--basepath"; + String ARG_COMPONENT = "--component"; + String ARG_COMPONENT_SHORT = "--comp"; + String ARG_COMPONENTS = "--components"; + String ARG_COMP_OPT= "--compopt"; + String ARG_COMP_OPT_SHORT = "--co"; + String ARG_CONFIG = "--config"; + String ARG_CONTAINERS = "--containers"; + String ARG_DEBUG = "--debug"; + String ARG_DEFINE = "-D"; + String ARG_DELETE = "--delete"; + String ARG_DEST = "--dest"; + String ARG_DESTDIR = "--destdir"; + String ARG_EXAMPLE = "--example"; + String ARG_EXAMPLE_SHORT = "-e"; + String ARG_FOLDER = "--folder"; + String ARG_FORCE = "--force"; + String ARG_FORMAT = "--format"; + String ARG_GETCONF = "--getconf"; + String ARG_GETEXP = "--getexp"; + String ARG_GETFILES = "--getfiles"; + String ARG_HELP = "--help"; + String ARG_IMAGE = "--image"; + String ARG_INSTALL = "--install"; + String ARG_INTERNAL = "--internal"; + String ARG_KEYLEN = "--keylen"; + String ARG_KEYTAB = "--keytab"; + String ARG_KEYTABINSTALL = ARG_INSTALL; + String ARG_KEYTABDELETE = ARG_DELETE; + String ARG_KEYTABLIST = "--list"; + String ARG_LIST = "--list"; + String ARG_LISTCONF = "--listconf"; + String ARG_LISTEXP = "--listexp"; + String ARG_LISTFILES = "--listfiles"; + String ARG_LIVE = "--live"; + String ARG_MANAGER = "--manager"; + String ARG_MANAGER_SHORT = "--m"; + String ARG_MESSAGE = "--message"; + String ARG_NAME = "--name"; + String ARG_OPTION = "--option"; + String ARG_OPTION_SHORT = "-O"; + String ARG_OUTPUT = "--out"; + String ARG_OUTPUT_SHORT = "-o"; + String ARG_OVERWRITE = "--overwrite"; + String ARG_PACKAGE = "--package"; + String ARG_PATH = "--path"; + String ARG_PRINCIPAL = "--principal"; + String ARG_QUEUE = "--queue"; + String ARG_SHORT_QUEUE = "-q"; + String ARG_LIFETIME = "--lifetime"; + String ARG_RESOURCE = "--resource"; + String ARG_RESOURCE_MANAGER = "--rm"; + String ARG_SECURE = "--secure"; + String ARG_SERVICETYPE = "--servicetype"; + String ARG_SERVICES = "--services"; + String ARG_SOURCE = "--source"; + String ARG_STATE = "--state"; + String ARG_SYSPROP = "-S"; + String ARG_USER = "--user"; + String ARG_UPLOAD = "--upload"; + String ARG_VERBOSE = "--verbose"; + String ARG_VERSION = "--version"; + String ARG_WAIT = "--wait"; +/* + STOP: DO NOT ADD YOUR ARGUMENTS HERE. GO BACK AND INSERT THEM IN THE + RIGHT PLACE IN THE LIST + */ + + // Tha path in hdfs to be read by Service AM + String ARG_SERVICE_DEF_PATH = "-cluster-uri"; + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ClientArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ClientArgs.java new file mode 100644 index 0000000..84d3bbd --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ClientArgs.java @@ -0,0 +1,193 @@ +/* + * 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.service.client.params; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.service.utils.SliderUtils; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.ErrorStrings; +import org.apache.hadoop.yarn.service.exceptions.SliderException; + +import java.util.Collection; + +/** + * Client CLI Args + */ + +public class ClientArgs extends CommonArgs { + + // ========================================================= + // Keep all of these in alphabetical order. Thanks. + // ========================================================= + + private final ActionBuildArgs actionBuildArgs = new ActionBuildArgs(); + private final ActionClientArgs actionClientArgs = new ActionClientArgs(); + private final ActionCreateArgs actionCreateArgs = new ActionCreateArgs(); + private final ActionDependencyArgs actionDependencyArgs = new ActionDependencyArgs(); + private final ActionDestroyArgs actionDestroyArgs = new ActionDestroyArgs(); + private final ActionExistsArgs actionExistsArgs = new ActionExistsArgs(); + private final ActionFlexArgs actionFlexArgs = new ActionFlexArgs(); + private final ActionFreezeArgs actionFreezeArgs = new ActionFreezeArgs(); + private final ActionHelpArgs actionHelpArgs = new ActionHelpArgs(); + private final ActionKeytabArgs actionKeytabArgs = new ActionKeytabArgs(); + private final ActionListArgs actionListArgs = new ActionListArgs(); + private final ActionRegistryArgs actionRegistryArgs = new ActionRegistryArgs(); + private final ActionResolveArgs actionResolveArgs = new ActionResolveArgs(); + private final ActionResourceArgs actionResourceArgs = new ActionResourceArgs(); + private final ActionStatusArgs actionStatusArgs = new ActionStatusArgs(); + private final ActionThawArgs actionThawArgs = new ActionThawArgs(); + private final ActionTokensArgs actionTokenArgs = new ActionTokensArgs(); + private final ActionUpdateArgs actionUpdateArgs = new ActionUpdateArgs(); + + public ClientArgs(String[] args) { + super(args); + } + + public ClientArgs(Collection args) { + super(args); + } + + @Override + protected void addActionArguments() { + + addActions( + actionBuildArgs, + actionCreateArgs, + actionDependencyArgs, + actionDestroyArgs, + actionFlexArgs, + actionFreezeArgs, + actionHelpArgs, + actionStatusArgs, + actionThawArgs + ); + } + + @Override + public void applyDefinitions(Configuration conf) throws + BadCommandArgumentsException { + super.applyDefinitions(conf); + } + + + public ActionBuildArgs getActionBuildArgs() { + return actionBuildArgs; + } + + public ActionUpdateArgs getActionUpdateArgs() { + return actionUpdateArgs; + } + + public ActionCreateArgs getActionCreateArgs() { + return actionCreateArgs; + } + + public ActionDependencyArgs getActionDependencyArgs() { + return actionDependencyArgs; + } + + public ActionFlexArgs getActionFlexArgs() { + return actionFlexArgs; + } + + /** + * Look at the chosen action and bind it as the core action for the operation. + * @throws SliderException bad argument or similar + */ + @Override + public void applyAction() throws SliderException { + String action = getAction(); + if (SliderUtils.isUnset(action)) { + action = ACTION_HELP; + } + switch (action) { + case ACTION_BUILD: + bindCoreAction(actionBuildArgs); + break; + + case ACTION_CREATE: + bindCoreAction(actionCreateArgs); + break; + + case ACTION_STOP: + bindCoreAction(actionFreezeArgs); + break; + + case ACTION_START: + bindCoreAction(actionThawArgs); + break; + + case ACTION_DEPENDENCY: + bindCoreAction(actionDependencyArgs); + break; + + case ACTION_DESTROY: + bindCoreAction(actionDestroyArgs); + break; + + case ACTION_EXISTS: + bindCoreAction(actionExistsArgs); + break; + + case ACTION_FLEX: + bindCoreAction(actionFlexArgs); + break; + + case ACTION_HELP: + bindCoreAction(actionHelpArgs); + break; + + case ACTION_KEYTAB: + bindCoreAction(actionKeytabArgs); + break; + + case ACTION_LIST: + bindCoreAction(actionListArgs); + break; + + case ACTION_REGISTRY: + bindCoreAction(actionRegistryArgs); + break; + + case ACTION_RESOLVE: + bindCoreAction(actionResolveArgs); + break; + + case ACTION_RESOURCE: + bindCoreAction(actionResourceArgs); + break; + + case ACTION_STATUS: + bindCoreAction(actionStatusArgs); + break; + + case ACTION_TOKENS: + bindCoreAction(actionTokenArgs); + break; + + case ACTION_UPDATE: + bindCoreAction(actionUpdateArgs); + break; + default: + throw new BadCommandArgumentsException(ErrorStrings.ERROR_UNKNOWN_ACTION + + " " + action); + } + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/CommonArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/CommonArgs.java new file mode 100644 index 0000000..145b44a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/CommonArgs.java @@ -0,0 +1,245 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParameterDescription; +import com.beust.jcommander.ParameterException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.service.utils.SliderUtils; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.ErrorStrings; +import org.apache.hadoop.yarn.service.exceptions.SliderException; +import org.apache.hadoop.yarn.service.exceptions.UsageException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * This class contains the common argument set for all tne entry points, + * and the core parsing logic to verify that the action is on the list + * of allowed actions -and that the remaining number of arguments is + * in the range allowed + */ + +public abstract class CommonArgs extends ArgOps implements SliderActions, + Arguments { + + protected static final Logger log = LoggerFactory.getLogger(CommonArgs.class); + + + private static final int DIFF_BETWEEN_DESCIPTION_AND_COMMAND_NAME = 30; + + + @Parameter(names = ARG_HELP, help = true) + public boolean help; + + + /** + -D name=value + + Define an HBase configuration option which overrides any options in + the configuration XML files of the image or in the image configuration + directory. The values will be persisted. + Configuration options are only passed to the cluster when creating or reconfiguring a cluster. + + */ + + public Map definitionMap = new HashMap(); + /** + * System properties + */ + public Map syspropsMap = new HashMap(); + + + /** + * fields + */ + public final JCommander commander; + private final String[] args; + + private AbstractActionArgs coreAction; + + /** + * get the name: relies on arg 1 being the cluster name in all operations + * @return the name argument, null if there is none + */ + public String getClusterName() { + return coreAction.getServiceName(); + } + + protected CommonArgs(String[] args) { + this.args = args; + commander = new JCommander(this); + } + + protected CommonArgs(Collection args) { + List argsAsStrings = SliderUtils.collectionToStringList(args); + this.args = argsAsStrings.toArray(new String[argsAsStrings.size()]); + commander = new JCommander(this); + } + + public String usage() { + return usage(this, null); + } + + public static String usage(CommonArgs serviceArgs, String commandOfInterest) { + String result = null; + StringBuilder helperMessage = new StringBuilder(); + if (commandOfInterest == null) { + // JCommander.usage is too verbose for a command with many options like + // slider no short version of that is found Instead, we compose our msg by + helperMessage.append("\nUsage: service COMMAND [options]\n"); + helperMessage.append("where COMMAND is one of\n"); + for (String jcommand : serviceArgs.commander.getCommands().keySet()) { + helperMessage.append(String.format("\t%-" + + DIFF_BETWEEN_DESCIPTION_AND_COMMAND_NAME + "s%s", jcommand, + serviceArgs.commander.getCommandDescription(jcommand) + "\n")); + } + helperMessage + .append("Most commands print help when invoked without parameters or with --help"); + result = helperMessage.toString(); + } else { + helperMessage.append("\nUsage: service ").append(commandOfInterest); + helperMessage.append(serviceArgs.coreAction.getMinParams() > 0 ? " " : ""); + helperMessage.append("\n"); + for (ParameterDescription paramDesc : serviceArgs.commander.getCommands() + .get(commandOfInterest).getParameters()) { + String optional = paramDesc.getParameter().required() ? " (required)" + : " (optional)"; + String paramName = paramDesc.getParameterized().getType() == Boolean.TYPE ? paramDesc + .getLongestName() : paramDesc.getLongestName() + " <" + + paramDesc.getParameterized().getName() + ">"; + helperMessage.append(String.format("\t%-" + + DIFF_BETWEEN_DESCIPTION_AND_COMMAND_NAME + "s%s", paramName, + paramDesc.getDescription() + optional + "\n")); + result = helperMessage.toString(); + } + } + return result; + } + + /** + * Parse routine -includes registering the action-specific argument classes + * and postprocess it + * @throws SliderException on any problem + */ + public void parse() throws SliderException { + addActionArguments(); + try { + commander.parse(args); + } catch (ParameterException e) { + commander.usage(args[0]); + throw e; + } + //now copy back to this class some of the attributes that are common to all + //actions + postProcess(); + } + + + protected void addActions(Object... actions) { + for (Object action : actions) { + commander.addCommand(action); + } + } + + /** + * Override point to add a set of actions + */ + protected void addActionArguments() { + + } + + /** + * validate args via {@link #validate()} + * then postprocess the arguments + */ + public void postProcess() throws SliderException { + applyAction(); + validate(); + + //apply entry set + for (Map.Entry entry : syspropsMap.entrySet()) { + System.setProperty(entry.getKey(), entry.getValue()); + } + } + + + /** + * Implementors must implement their action apply routine here + */ + public abstract void applyAction() throws SliderException; + + + /** + * Bind the core action; this extracts any attributes that are used + * across routines + * @param action action to bind + */ + protected void bindCoreAction(AbstractActionArgs action) { + coreAction = action; + + splitPairs(coreAction.definitions, definitionMap); + splitPairs(coreAction.sysprops, syspropsMap); + } + + /** + * Validate the arguments against the action requested + */ + public void validate() throws BadCommandArgumentsException, UsageException { + if (coreAction == null) { + throw new UsageException(ErrorStrings.ERROR_NO_ACTION + usage()); + } + log.debug("action={}", getAction()); + // let the action validate itself + try { + coreAction.validate(); + } catch (BadCommandArgumentsException e) { + String badArgMsgBuilder = + e.getMessage() + System.lineSeparator() + usage(this, + coreAction.getActionName()); + throw new BadCommandArgumentsException(badArgMsgBuilder); + } + } + + /** + * Apply all the definitions on the command line to the configuration + * @param conf config + */ + public void applyDefinitions(Configuration conf) throws + BadCommandArgumentsException { + applyDefinitions(definitionMap, conf); + } + + public boolean isDebug() { + return coreAction.debug; + } + + + public String getAction() { + return commander.getParsedCommand(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ComponentArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ComponentArgsDelegate.java new file mode 100644 index 0000000..b6cd0a1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ComponentArgsDelegate.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.service.client.params; + +import com.beust.jcommander.Parameter; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class ComponentArgsDelegate extends AbstractArgsDelegate { + + /** + * This is a listing of the roles to create + */ + @Parameter(names = {ARG_COMPONENT, ARG_COMPONENT_SHORT}, + arity = 2, + description = "--component e.g. +1 incr by 1, -2 decr by 2, and 3 makes final count 3", + splitter = DontSplitArguments.class) + public List componentTuples = new ArrayList<>(0); + + + /** + * Get the role mapping (may be empty, but never null) + * @return role mapping + * @throws BadCommandArgumentsException parse problem + */ + public Map getComponentMap() throws BadCommandArgumentsException { + return convertTupleListToMap("component", componentTuples); + } + + public List getComponentTuples() { + return componentTuples; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/DontSplitArguments.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/DontSplitArguments.java new file mode 100644 index 0000000..85de615 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/DontSplitArguments.java @@ -0,0 +1,34 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.converters.IParameterSplitter; + +import java.util.ArrayList; +import java.util.List; + +public class DontSplitArguments implements IParameterSplitter { + + @Override + public List split(String value) { + List list = new ArrayList<>(1); + list.add(value); + return list; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/LaunchArgsAccessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/LaunchArgsAccessor.java new file mode 100644 index 0000000..bf194b6 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/LaunchArgsAccessor.java @@ -0,0 +1,30 @@ +/* + * 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.service.client.params; + +import java.io.File; + +/** + * Launch args for create and start and anything else that can start something + */ +public interface LaunchArgsAccessor extends WaitTimeAccessor { + String getRmAddress(); + + File getOutputFile(); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/LaunchArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/LaunchArgsDelegate.java new file mode 100644 index 0000000..d42510c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/LaunchArgsDelegate.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.service.client.params; + +import com.beust.jcommander.Parameter; + +import java.io.File; + +/** + * Any launch-time args + */ +public class LaunchArgsDelegate extends WaitArgsDelegate implements + LaunchArgsAccessor { + + + //TODO: do we need this? + @Parameter(names = ARG_RESOURCE_MANAGER, + description = "Resource manager hostname:port ", + required = false) + private String rmAddress; + + @Override + public String getRmAddress() { + return rmAddress; + } + + @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT}, + description = "output file for any service report") + public File outputFile; + + @Override + public File getOutputFile() { + return outputFile; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/OptionArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/OptionArgsDelegate.java new file mode 100644 index 0000000..7972716 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/params/OptionArgsDelegate.java @@ -0,0 +1,66 @@ +/* + * 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.service.client.params; + +import com.beust.jcommander.Parameter; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Delegate for application and resource options. + */ +public class OptionArgsDelegate extends AbstractArgsDelegate { + + /** + * Options key value. + */ + @Parameter(names = {ARG_OPTION, ARG_OPTION_SHORT}, arity = 2, + description = ARG_OPTION + " ", + splitter = DontSplitArguments.class) + public List optionTuples = new ArrayList<>(0); + + + /** + * All the app component option triples. + */ + @Parameter(names = {ARG_COMP_OPT, ARG_COMP_OPT_SHORT}, arity = 3, + description = "Component option " + ARG_COMP_OPT + + "