diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-slider-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-services-dist.xml similarity index 100% rename from hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-slider-dist.xml rename to hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-services-dist.xml diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn index 98a2036..ac8fc34 100755 --- a/hadoop-yarn-project/hadoop-yarn/bin/yarn +++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn @@ -47,7 +47,7 @@ function hadoop_usage hadoop_add_subcommand "resourcemanager" "run the ResourceManager" hadoop_add_subcommand "rmadmin" "admin tools" hadoop_add_subcommand "scmadmin" "SharedCacheManager admin tools" - hadoop_add_subcommand "servicesapi" "run yarn-service rest server" + hadoop_add_subcommand "servicesapi" "run yarn-native-service rest server" hadoop_add_subcommand "sharedcachemanager" "run the SharedCacheManager daemon" hadoop_add_subcommand "service" "run a service" hadoop_add_subcommand "timelinereader" "run the timeline reader server" @@ -146,7 +146,7 @@ function yarncmd_case HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true" hadoop_add_classpath "${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/slider"'/*' hadoop_add_classpath "${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/services-api"'/*' - HADOOP_CLASSNAME='org.apache.hadoop.yarn.services.webapp.ApplicationApiWebApp' + HADOOP_CLASSNAME='org.apache.hadoop.yarn.service.webapp.ApplicationApiWebApp' local sld="${HADOOP_YARN_HOME}/${YARN_DIR},\ ${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR},\ ${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/slider,\ @@ -155,7 +155,7 @@ ${HADOOP_HDFS_HOME}/${HDFS_LIB_JARS_DIR},\ ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_DIR},\ ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}" hadoop_translate_cygwin_path sld - hadoop_add_param HADOOP_OPTS slider.libdir "-Dslider.libdir=${sld}" + hadoop_add_param HADOOP_OPTS service.libdir "-Dservice.libdir=${sld}" ;; sharedcachemanager) HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true" @@ -172,7 +172,7 @@ ${HADOOP_HDFS_HOME}/${HDFS_LIB_JARS_DIR},\ ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_DIR},\ ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}" hadoop_translate_cygwin_path sld - hadoop_add_param HADOOP_OPTS slider.libdir "-Dslider.libdir=${sld}" + hadoop_add_param HADOOP_OPTS service.libdir "-Dservice.libdir=${sld}" ;; timelinereader) HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true" diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApplicationApiService.java similarity index 79% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApplicationApiService.java index decd849..30fc5ef 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApplicationApiService.java @@ -15,23 +15,21 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.services.api.impl; +package org.apache.hadoop.yarn.service.webapp; 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.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType; 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.client.ServiceClient; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.ApplicationState; -import org.apache.slider.api.resource.ApplicationStatus; -import org.apache.slider.api.resource.Component; -import org.apache.slider.common.tools.SliderUtils; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.api.records.ApplicationState; +import org.apache.hadoop.yarn.service.api.records.ApplicationStatus; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.utils.SliderUtils; import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,12 +49,10 @@ import java.util.Collections; import java.util.Map; -import static org.apache.slider.util.RestApiConstants.*; +import static org.apache.hadoop.yarn.service.conf.RestApiConstants.*; @Singleton -@Path(APPLICATIONS_API_RESOURCE_PATH) -@Consumes({ MediaType.APPLICATION_JSON }) -@Produces({ MediaType.APPLICATION_JSON }) +@Path(CONTEXT_ROOT) public class ApplicationApiService { private static final Logger LOG = LoggerFactory.getLogger(ApplicationApiService.class); @@ -75,15 +71,18 @@ private static void init() { } @GET - @Path("/versions/yarn-service-version") + @Path(VERSION) @Consumes({ MediaType.APPLICATION_JSON }) - @Produces({ MediaType.APPLICATION_JSON }) public Response getSliderVersion() { + @Produces({ MediaType.APPLICATION_JSON, MediaType.TEXT_PLAIN }) + public Response getVersion() { String version = VersionInfo.getBuildVersion(); LOG.info(version); return Response.ok(version).build(); } - @POST @Consumes({ MediaType.APPLICATION_JSON }) + @POST + @Path(APP_ROOT_PATH) + @Consumes({ MediaType.APPLICATION_JSON }) @Produces({ MediaType.APPLICATION_JSON }) public Response createApplication(Application application) { LOG.info("POST: createApplication = {}", application); @@ -94,7 +93,7 @@ public Response createApplication(Application application) { + " applicationId = " + applicationId); applicationStatus.setState(ApplicationState.ACCEPTED); applicationStatus.setUri( - CONTEXT_ROOT + APPLICATIONS_API_RESOURCE_PATH + "/" + application + CONTEXT_ROOT + APP_ROOT_PATH + "/" + application .getName()); return Response.status(Status.CREATED).entity(applicationStatus).build(); } catch (IllegalArgumentException e) { @@ -110,10 +109,11 @@ public Response createApplication(Application application) { } } - @GET @Path("/{app_name}") + @GET + @Path(APP_PATH) @Consumes({ MediaType.APPLICATION_JSON }) @Produces({ MediaType.APPLICATION_JSON }) - public Response getApplication(@PathParam("app_name") String appName) { + public Response getApplication(@PathParam(APP_NAME) String appName) { LOG.info("GET: getApplication for appName = {}", appName); ApplicationStatus applicationStatus = new ApplicationStatus(); @@ -127,22 +127,7 @@ public Response getApplication(@PathParam("app_name") String appName) { try { Application app = SERVICE_CLIENT.getStatus(appName); - ApplicationReport report = SERVICE_CLIENT.getYarnClient() - .getApplicationReport(ApplicationId.fromString(app.getId())); - if (report != null) { - app.setLifetime( - report.getApplicationTimeouts().get(ApplicationTimeoutType.LIFETIME) - .getRemainingTime()); - LOG.info("Application = {}", app); - return Response.ok(app).build(); - } else { - String message = "Application " + appName + " does not exist."; - LOG.info(message); - applicationStatus.setCode(ERROR_CODE_APP_DOES_NOT_EXIST); - applicationStatus.setDiagnostics(message); - return Response.status(Status.NOT_FOUND).entity(applicationStatus) - .build(); - } + return Response.ok(app).build(); } catch (Exception e) { LOG.error("Get application failed", e); applicationStatus @@ -153,17 +138,17 @@ public Response getApplication(@PathParam("app_name") String appName) { } @DELETE - @Path("/{app_name}") + @Path(APP_PATH) @Consumes({ MediaType.APPLICATION_JSON }) @Produces({ MediaType.APPLICATION_JSON }) - public Response deleteApplication(@PathParam("app_name") String appName) { + public Response deleteApplication(@PathParam(APP_NAME) String appName) { LOG.info("DELETE: deleteApplication for appName = {}", appName); return stopApplication(appName, true); } private Response stopApplication(String appName, boolean destroy) { try { - SERVICE_CLIENT.actionStop(appName); + SERVICE_CLIENT.actionStop(appName, destroy); if (destroy) { SERVICE_CLIENT.actionDestroy(appName); LOG.info("Successfully deleted application {}", appName); @@ -185,11 +170,12 @@ private Response stopApplication(String appName, boolean destroy) { } } - @PUT @Path("/{app_name}/components/{component_name}") + @PUT + @Path(COMPONENT_PATH) @Consumes({ MediaType.APPLICATION_JSON }) - @Produces({ MediaType.APPLICATION_JSON }) - public Response updateComponent(@PathParam("app_name") String appName, - @PathParam("component_name") String componentName, Component component) { + @Produces({ MediaType.APPLICATION_JSON, MediaType.TEXT_PLAIN }) + public Response updateComponent(@PathParam(APP_NAME) String appName, + @PathParam(COMPONENT_NAME) String componentName, Component component) { if (component.getNumberOfContainers() < 0) { return Response.status(Status.BAD_REQUEST).entity( @@ -201,9 +187,10 @@ public Response updateComponent(@PathParam("app_name") String appName, Map original = SERVICE_CLIENT.flexByRestService(appName, Collections.singletonMap(component.getName(), component.getNumberOfContainers())); - return Response.ok().entity("Updating " + componentName + " size from " - + original.get(componentName) + " to " - + component.getNumberOfContainers()).build(); + return Response.ok().entity( + "Updating component " + componentName + " size from " + original + .get(componentName) + " to " + component.getNumberOfContainers()) + .build(); } catch (YarnException | IOException e) { ApplicationStatus status = new ApplicationStatus(); status.setDiagnostics(e.getMessage()); @@ -212,10 +199,11 @@ public Response updateComponent(@PathParam("app_name") String appName, } } - @PUT @Path("/{app_name}") + @PUT + @Path(APP_PATH) @Consumes({ MediaType.APPLICATION_JSON }) @Produces({ MediaType.APPLICATION_JSON }) - public Response updateApplication(@PathParam("app_name") String appName, + public Response updateApplication(@PathParam(APP_NAME) String appName, Application updateAppData) { LOG.info("PUT: updateApplication for app = {} with data = {}", appName, updateAppData); @@ -246,8 +234,7 @@ public Response updateApplication(@PathParam("app_name") String appName, // flex a single component app if (updateAppData.getNumberOfContainers() != null && !ServiceApiUtil - .hasComponent( - updateAppData)) { + .hasComponent(updateAppData)) { Component defaultComp = ServiceApiUtil.createDefaultComponent(updateAppData); return updateComponent(updateAppData.getName(), defaultComp.getName(), defaultComp); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApplicationApiWebApp.java similarity index 94% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApplicationApiWebApp.java index 7fc01a1..7225209 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApplicationApiWebApp.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.services.webapp; +package org.apache.hadoop.yarn.service.webapp; -import static org.apache.slider.util.RestApiConstants.*; +import static org.apache.hadoop.yarn.service.conf.RestApiConstants.*; import java.io.IOException; import java.net.InetAddress; @@ -27,7 +27,6 @@ import org.apache.commons.lang.StringUtils; import org.apache.hadoop.http.HttpServer2; import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.yarn.services.api.impl.ApplicationApiService; import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider; import org.eclipse.jetty.webapp.Configuration; @@ -100,8 +99,7 @@ protected void startWebApp() throws IOException { ApplicationApiService.class.getPackage().getName() + SEP + GenericExceptionHandler.class.getPackage().getName() + SEP + YarnJacksonJaxbJsonProvider.class.getPackage().getName(); - applicationApiServer.addJerseyResourcePackage(apiPackages, CONTEXT_ROOT - + "/*"); + applicationApiServer.addJerseyResourcePackage(apiPackages, "/*"); try { logger.info("Application starting up. Logging start..."); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/ApplicationApi.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/ApplicationApi.java deleted file mode 100644 index 0f4bdae..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/ApplicationApi.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.services.api; - -import javax.ws.rs.core.Response; - -import org.apache.slider.api.resource.Application; - -/** - * Apache Hadoop YARN Services REST API interface. - * - */ -public interface ApplicationApi { - Response createApplication(Application application); - - Response getApplications(String state); - - Response deleteApplication(String appName); - - Response updateApplication(String appName, Application updateAppData); -} 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 index 2f090a2..88f74ef 100644 --- 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 @@ -277,9 +277,6 @@ definitions: type: integer format: int64 description: Number of containers for this app-component (optional). If not specified, the application level global number_of_containers takes effect. - unique_component_support: - type: boolean - description: Certain applications need to define multiple components using the same artifact and resource profile, differing only in configurations. In such cases, this field helps app owners to avoid creating multiple component definitions with repeated information. The number_of_containers field dictates the initial number of components created. Component names typically differ with a trailing id, but assumptions should not be made on that, as the algorithm can change at any time. Configurations section will be able to use placeholders like ${USER}, ${CLUSTER_NAME} and ${COMPONENT_NAME} to be replaced at runtime with user the app is submitted as, application name and application component name respectively. Launch command can use placeholders like ${APP_COMPONENT_NAME} and ${APP_NAME} to get its component name and app name respectively at runtime. The best part of this feature is that when the component is flexed up, entirely new components (with new trailing ids) are created. run_privileged_container: type: boolean description: Run all containers of this component in privileged mode (YARN-4262). 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 index 31e3051..1282c9f 100644 --- 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 @@ -21,7 +21,7 @@ com.sun.jersey.spi.container.servlet.ServletContainer com.sun.jersey.config.property.packages - org.apache.hadoop.yarn.services.webapp,org.apache.hadoop.yarn.services.api,org.apache.slider.api.resource,org.apache.hadoop.yarn.services.api.impl + 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 diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/dev-support/findbugs-exclude.xml index f2bf582..2814cca 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/dev-support/findbugs-exclude.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/dev-support/findbugs-exclude.xml @@ -23,99 +23,26 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + - - + + - + - - - + + - - - - - - - - + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/conf/slideram-log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/conf/yarnservice-log4j.properties similarity index 82% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/conf/slideram-log4j.properties rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/conf/yarnservice-log4j.properties index 333859e..58c8e27 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/conf/slideram-log4j.properties +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/conf/yarnservice-log4j.properties @@ -17,14 +17,14 @@ # This is the log4j configuration for Slider Application Master -# Log rotation based on size (100KB) with a max of 10 backup files +# 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}/slider.log -log4j.appender.amlog.MaxFileSize=1MB -log4j.appender.amlog.MaxBackupIndex=10 +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 @@ -43,13 +43,8 @@ log4j.appender.stderr.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} - %m% log4j.appender.subprocess=org.apache.log4j.ConsoleAppender log4j.appender.subprocess.layout=org.apache.log4j.PatternLayout log4j.appender.subprocess.layout.ConversionPattern=[%c{1}]: %m%n -#log4j.logger.org.apache.slider.yarn.appmaster.SliderAppMasterer.master=INFO,subprocess -# for debugging Slider -#log4j.logger.org.apache.slider=DEBUG - -# uncomment to debug service lifecycle issues -#log4j.logger.org.apache.hadoop.yarn.service.launcher=DEBUG +# for debugging yarn-service framework #log4j.logger.org.apache.hadoop.yarn.service=DEBUG # uncomment for YARN operations @@ -61,7 +56,6 @@ log4j.appender.subprocess.layout.ConversionPattern=[%c{1}]: %m%n #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.zookeeper=WARN log4j.logger.org.apache.curator.framework.state=ERROR diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml index 02317e5..c8de037 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml @@ -59,8 +59,6 @@ ${basedir}/src/main/proto - SliderClusterMessages.proto - SliderClusterProtocol.proto ClientAMProtocol.proto @@ -92,32 +90,11 @@ - org.apache.avro - avro-maven-plugin - ${avro.version} - - - generate-sources - - schema - - - ${project.basedir}/src/main/avro/ - - ${project.build.directory}/generated-sources/java - - - - - - - org.apache.rat apache-rat-plugin **/*.json - src/main/resources/webapps/slideram/.keep @@ -218,11 +195,6 @@ - org.apache.avro - avro - - - org.apache.commons commons-compress @@ -422,7 +394,7 @@ false ${project.artifactId}-${project.version} - hadoop-yarn-slider-dist + hadoop-yarn-services-dist diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/avro/org/apache/slider/server/avro/RoleHistoryRecord.avsc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/avro/org/apache/slider/server/avro/RoleHistoryRecord.avsc deleted file mode 100644 index 3667c01..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/avro/org/apache/slider/server/avro/RoleHistoryRecord.avsc +++ /dev/null @@ -1,114 +0,0 @@ -// 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. - -[ - - { - "type": "record", - "name": "NodeEntryRecord", - "namespace": "org.apache.slider.server.avro", - "fields": [ - { - "name": "host", - "type": "string" - }, - { - "name": "role", - "type": "int" - }, - { - "name": "active", - "type": "boolean" - }, - { - "name": "last_used", - "type": "long" - } - ] - }, - - { - "type": "record", - "name": "RoleHistoryHeader", - "namespace": "org.apache.slider.server.avro", - "fields": [ - { - "name": "version", - "type": "int" - }, - { - "name": "saved", - "type": "long" - }, - { - "name": "savedx", - "type": "string" - }, - { - "name": "savedate", - "type": "string", - "default": "" - }, - { - "name": "roles", - "type": "int" - } - ] - }, - { - "type": "record", - "name": "RoleHistoryMapping", - "namespace": "org.apache.slider.server.avro", - "fields": [ - { - "name": "rolemap", - "type": { - "type": "map", - "values": "int" - } - } - ] - }, - { - "type": "record", - "name": "RoleHistoryFooter", - "namespace": "org.apache.slider.server.avro", - "fields": [ - { - "name": "count", - "type": "long" - } - ] - }, - - { - "type": "record", - "name": "RoleHistoryRecord", - "namespace": "org.apache.slider.server.avro", - "fields": [ - { - "name": "entry", - "type": [ - "org.apache.slider.server.avro.NodeEntryRecord", - "org.apache.slider.server.avro.RoleHistoryHeader", - "org.apache.slider.server.avro.RoleHistoryFooter", - "org.apache.slider.server.avro.RoleHistoryMapping" - ] - } - ] - } - -] diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ContainerFailureTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ContainerFailureTracker.java index bbb4c44..4743f28 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ContainerFailureTracker.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ContainerFailureTracker.java @@ -29,7 +29,7 @@ import java.util.Map; import java.util.Set; -import static org.apache.slider.api.ResourceKeys.NODE_FAILURE_THRESHOLD; +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.NODE_BLACKLIST_THRESHOLD; /** * This tracks the container failures per node. If the failure counter exceeds @@ -52,7 +52,7 @@ public ContainerFailureTracker(ServiceContext context, Component component) { this.context = context; this.component = component; maxFailurePerNode = component.getComponentSpec().getConfiguration() - .getPropertyInt(NODE_FAILURE_THRESHOLD, 3); + .getPropertyInt(NODE_BLACKLIST_THRESHOLD, 3); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceContext.java index 80668a0..c7616af 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceContext.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceContext.java @@ -21,9 +21,9 @@ 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.slider.api.resource.Application; -import org.apache.slider.api.resource.ConfigFile; -import org.apache.slider.common.tools.SliderFileSystem; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.api.records.ConfigFile; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; public class ServiceContext { public Application application = null; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java index c22dec4..d099f8c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java @@ -35,10 +35,12 @@ import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager; import org.apache.hadoop.yarn.service.client.params.SliderAMArgs; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.servicemonitor.ServiceMonitor; import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadClusterStateException; +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; @@ -68,6 +70,7 @@ protected void serviceInit(Configuration conf) throws Exception { context = new ServiceContext(); Path appDir = getAppDir(); + context.serviceHdfsDir = appDir.toString(); SliderFileSystem fs = new SliderFileSystem(conf); context.fs = fs; fs.setAppDir(appDir); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java index 590655f..8c968dc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java @@ -54,26 +54,25 @@ 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.constants.ServiceApiConstants; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.api.records.ConfigFile; import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; import org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEvent; import org.apache.hadoop.yarn.service.compinstance.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.SliderKeys; +import org.apache.hadoop.yarn.service.conf.YarnServiceConstants; +import org.apache.hadoop.yarn.service.containerlaunch.ContainerLaunchService; import org.apache.hadoop.yarn.service.metrics.ServiceMetrics; 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.apache.slider.api.RoleKeys; -import org.apache.slider.api.ServiceApiConstants; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.ConfigFile; -import org.apache.slider.core.registry.info.CustomRegistryConstants; -import org.apache.slider.core.zk.ZKIntegration; -import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,8 +92,8 @@ 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.constants.ServiceApiConstants.*; import static org.apache.hadoop.yarn.service.component.ComponentEventType.*; -import static org.apache.slider.api.ServiceApiConstants.*; /** * @@ -203,7 +202,7 @@ public void buildInstance(ServiceContext context, Configuration configuration) protected YarnRegistryViewForProviders createYarnRegistryOperations( ServiceContext context, RegistryOperations registryClient) { return new YarnRegistryViewForProviders(registryClient, - RegistryUtils.currentUser(), SliderKeys.APP_TYPE, app.getName(), + RegistryUtils.currentUser(), YarnServiceConstants.APP_TYPE, app.getName(), context.attemptId); } @@ -236,7 +235,8 @@ public void serviceStop() throws Exception { DefaultMetricsSystem.shutdown(); if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) { - serviceTimelinePublisher.serviceAttemptUnregistered(context); + serviceTimelinePublisher + .serviceAttemptUnregistered(context, diagnostics.toString()); } // Cleanup each component instance. no need to release containers as // they will be automatically released by RM @@ -285,7 +285,7 @@ private void initGlobalTokensForSubstitute(ServiceContext context) { LOG.error("Failed to get user.", e); } globalTokens - .put(SERVICE_ZK_PATH, ZKIntegration.mkClusterPath(user, app.getName())); + .put(SERVICE_ZK_PATH, ServiceRegistryUtils.mkClusterPath(user, app.getName())); globalTokens.put(ServiceApiConstants.USER, user); String dnsDomain = getConfig().getTrimmed(KEY_DNS_DOMAIN); @@ -336,7 +336,7 @@ private void createConfigFileCache(final FileSystem fileSystem) { context.configCache = configFileCache; } - protected void registerServiceInstance(ApplicationAttemptId attemptId, + private void registerServiceInstance(ApplicationAttemptId attemptId, Application application) throws IOException { LOG.info("Registering " + attemptId + ", " + application.getName() + " into registry"); @@ -345,11 +345,11 @@ protected void registerServiceInstance(ApplicationAttemptId attemptId, attemptId.getApplicationId().toString()); serviceRecord.set(YarnRegistryAttributes.YARN_PERSISTENCE, PersistencePolicies.APPLICATION); - serviceRecord.description = "Slider Application Master"; + serviceRecord.description = "Yarn Service Master"; serviceRecord.addExternalEndpoint(RegistryTypeUtils - .ipcEndpoint(CustomRegistryConstants.AM_IPC_PROTOCOL, - new InetSocketAddress(5000))); // FIXME + .ipcEndpoint("classpath:org.apache.hadoop.yarn.service.appmaster.ipc", + context.clientAMService.getBindAddress())); // set any provided attributes setUserProvidedServiceRecordAttributes(application.getConfiguration(), @@ -376,13 +376,13 @@ protected void registerServiceInstance(ApplicationAttemptId attemptId, } }); if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) { - serviceTimelinePublisher.serviceAttemptRegistered(app); + serviceTimelinePublisher.serviceAttemptRegistered(app, getConfig()); } } private void setUserProvidedServiceRecordAttributes( - org.apache.slider.api.resource.Configuration conf, ServiceRecord record) { - String prefix = RoleKeys.SERVICE_RECORD_ATTRIBUTE_PREFIX; + 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); @@ -395,10 +395,10 @@ private void createAllComponents() { long allocateId = 0; // sort components by dependencies - Collection sortedComponents = + Collection sortedComponents = ServiceApiUtil.sortByDependencies(app.getComponents()); - for (org.apache.slider.api.resource.Component compSpec : sortedComponents) { + 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); @@ -517,7 +517,7 @@ public void onContainersUpdated(List containers) { @Override public float getProgress() { // get running containers over desired containers long total = 0; - for (org.apache.slider.api.resource.Component component : app + for (org.apache.hadoop.yarn.service.api.records.Component component : app .getComponents()) { total += component.getNumberOfContainers(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ServiceApiConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/constants/ServiceApiConstants.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ServiceApiConstants.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/constants/ServiceApiConstants.java index fa21211..cf9e31f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ServiceApiConstants.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/constants/ServiceApiConstants.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.api; +package org.apache.hadoop.yarn.service.api.constants; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Application.java similarity index 98% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Application.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Application.java index 626efb8..f9e5154 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Application.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Application.java @@ -16,13 +16,15 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +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; @@ -36,7 +38,8 @@ /** * An Application resource has the following attributes. **/ - +@InterfaceAudience.Public +@InterfaceStability.Unstable @ApiModel(description = "An Application 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 diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ApplicationState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ApplicationState.java similarity index 83% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ApplicationState.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ApplicationState.java index 6827c16..acef562 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ApplicationState.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ApplicationState.java @@ -15,14 +15,17 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +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 application. **/ - +@InterfaceAudience.Public +@InterfaceStability.Unstable @ApiModel(description = "The current state of an application.") @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") public enum ApplicationState { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ApplicationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ApplicationStatus.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ApplicationStatus.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ApplicationStatus.java index 06960a8..b57225a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ApplicationStatus.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ApplicationStatus.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +package org.apache.hadoop.yarn.service.api.records; import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; @@ -26,12 +26,15 @@ 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 application, returned as a response to the * GET API. **/ - +@InterfaceAudience.Public +@InterfaceStability.Unstable @ApiModel(description = "The current status of a submitted application, 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 diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Artifact.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Artifact.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Artifact.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Artifact.java index f274d7d..0ddc374 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Artifact.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Artifact.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +package org.apache.hadoop.yarn.service.api.records; import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; @@ -26,11 +26,14 @@ 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 application component. **/ - +@InterfaceAudience.Public +@InterfaceStability.Unstable @ApiModel(description = "Artifact of an application 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) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/BaseResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/BaseResource.java similarity index 86% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/BaseResource.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/BaseResource.java index a23c1fb..a87c97f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/BaseResource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/BaseResource.java @@ -15,10 +15,14 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +package org.apache.hadoop.yarn.service.api.records; -import java.io.Serializable; +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; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Component.java similarity index 82% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Component.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Component.java index c15f82c..633e862 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Component.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Component.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +package org.apache.hadoop.yarn.service.api.records; import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; @@ -31,6 +31,8 @@ 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 application. If the application is HBase say, @@ -39,7 +41,8 @@ * applications say Kafka or Storm. Thereby it opens up the support for complex * and nested applications. **/ - +@InterfaceAudience.Public +@InterfaceStability.Unstable @ApiModel(description = "One or more components of the application. If the application is HBase say, then the component can be a simple role like master or regionserver. If the application is a complex business webapp then a component can be other applications say Kafka or Storm. Thereby it opens up the support for complex and nested applications.") @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") @XmlRootElement @@ -54,7 +57,6 @@ private String launchCommand = null; private Resource resource = null; private Long numberOfContainers = null; - private Boolean uniqueComponentSupport = false; private Boolean runPrivilegedContainer = false; private PlacementPolicy placementPolicy = null; private Configuration configuration = new Configuration(); @@ -226,38 +228,6 @@ public Container getContainer(String id) { } /** - * Certain applications need to define multiple components using the same - * artifact and resource profile, differing only in configurations. In such - * cases, this field helps app owners to avoid creating multiple component - * definitions with repeated information. The number_of_containers field - * dictates the initial number of components created. Component names - * typically differ with a trailing id, but assumptions should not be made on - * that, as the algorithm can change at any time. Configurations section will - * be able to use placeholders like ${USER}, ${CLUSTER_NAME} and - * ${COMPONENT_NAME} to be replaced at runtime with user the app is submitted - * as, application name and application component name respectively. Launch - * command can use placeholders like ${APP_COMPONENT_NAME} and ${APP_NAME} to - * get its component name and app name respectively at runtime. The best part - * of this feature is that when the component is flexed up, entirely new - * components (with new trailing ids) are created. - **/ - public Component uniqueComponentSupport(Boolean uniqueComponentSupport) { - this.uniqueComponentSupport = uniqueComponentSupport; - return this; - } - - @ApiModelProperty(example = "null", value = "Certain applications need to define multiple components using the same artifact and resource profile, differing only in configurations. In such cases, this field helps app owners to avoid creating multiple component definitions with repeated information. The number_of_containers field dictates the initial number of components created. Component names typically differ with a trailing id, but assumptions should not be made on that, as the algorithm can change at any time. Configurations section will be able to use placeholders like ${USER}, ${CLUSTER_NAME} and ${COMPONENT_NAME} to be replaced at runtime with user the app is submitted as, application name and application component name respectively. Launch command can use placeholders like ${APP_COMPONENT_NAME} and ${APP_NAME} to get its component name and app name respectively at runtime. The best part of this feature is that when the component is flexed up, entirely new components (with new trailing ids) are created.") - @JsonProperty("unique_component_support") - public Boolean getUniqueComponentSupport() { - return uniqueComponentSupport; - } - - @XmlElement(name = "unique_component_support") - public void setUniqueComponentSupport(Boolean uniqueComponentSupport) { - this.uniqueComponentSupport = uniqueComponentSupport; - } - - /** * Run all containers of this component in privileged mode (YARN-4262). **/ public Component runPrivilegedContainer(Boolean runPrivilegedContainer) { @@ -351,8 +321,6 @@ public boolean equals(java.lang.Object o) { && Objects.equals(this.launchCommand, component.launchCommand) && Objects.equals(this.resource, component.resource) && Objects.equals(this.numberOfContainers, component.numberOfContainers) - && Objects.equals(this.uniqueComponentSupport, - component.uniqueComponentSupport) && Objects.equals(this.runPrivilegedContainer, component.runPrivilegedContainer) && Objects.equals(this.placementPolicy, component.placementPolicy) @@ -363,7 +331,7 @@ public boolean equals(java.lang.Object o) { @Override public int hashCode() { return Objects.hash(name, dependencies, readinessCheck, artifact, - launchCommand, resource, numberOfContainers, uniqueComponentSupport, + launchCommand, resource, numberOfContainers, runPrivilegedContainer, placementPolicy, configuration, quicklinks); } @@ -385,8 +353,6 @@ public String toString() { .append(toIndentedString(numberOfContainers)).append("\n"); sb.append(" containers: ").append(toIndentedString(containers)) .append("\n"); - sb.append(" uniqueComponentSupport: ") - .append(toIndentedString(uniqueComponentSupport)).append("\n"); sb.append(" runPrivilegedContainer: ") .append(toIndentedString(runPrivilegedContainer)).append("\n"); sb.append(" placementPolicy: ").append(toIndentedString(placementPolicy)) @@ -433,9 +399,6 @@ public void mergeFrom(Component that) { if (this.getRunPrivilegedContainer() == null) { this.setRunPrivilegedContainer(that.getRunPrivilegedContainer()); } - if (this.getUniqueComponentSupport() == null) { - this.setUniqueComponentSupport(that.getUniqueComponentSupport()); - } if (this.getDependencies() == null) { this.setDependencies(that.getDependencies()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ConfigFile.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFile.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ConfigFile.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFile.java index b4040b6..2fb494e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ConfigFile.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFile.java @@ -15,13 +15,15 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +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.XmlRootElement; @@ -34,7 +36,8 @@ * A config file that needs to be created and made available as a volume in an * application component container. **/ - +@InterfaceAudience.Public +@InterfaceStability.Unstable @ApiModel(description = "A config file that needs to be created and made available as a volume in an application component container.") @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") @XmlRootElement diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFormat.java similarity index 87% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFormat.java index 081688b..e10305a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFormat.java @@ -16,10 +16,15 @@ * limitations under the License. */ -package org.apache.slider.core.registry.docstore; +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"), diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Configuration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Configuration.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Configuration.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Configuration.java index e89306c..0ac508b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Configuration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Configuration.java @@ -15,14 +15,16 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +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.slider.common.tools.SliderUtils; +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; @@ -37,7 +39,8 @@ * 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 application 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) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Container.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Container.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Container.java index c5dc627..8b687bb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Container.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Container.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +package org.apache.hadoop.yarn.service.api.records; import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; @@ -28,11 +28,14 @@ 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 application container. **/ - +@InterfaceAudience.Public +@InterfaceStability.Unstable @ApiModel(description = "An instance of a running application container") @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00") @XmlRootElement diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ContainerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ContainerState.java similarity index 80% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ContainerState.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ContainerState.java index cd1ef4a..bf09ff2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ContainerState.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ContainerState.java @@ -15,11 +15,16 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +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-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Error.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Error.java similarity index 93% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Error.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Error.java index 3cf9b29..c64b1b5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Error.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Error.java @@ -15,14 +15,18 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +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 { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/PlacementPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/PlacementPolicy.java similarity index 93% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/PlacementPolicy.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/PlacementPolicy.java index 306338f..7d1b889 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/PlacementPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/PlacementPolicy.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +package org.apache.hadoop.yarn.service.api.records; import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; @@ -24,12 +24,15 @@ 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 application. This feature is in the * works in YARN-4902. **/ - +@InterfaceAudience.Public +@InterfaceStability.Unstable @ApiModel(description = "Placement policy of an instance of an application. 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 { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ReadinessCheck.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ReadinessCheck.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ReadinessCheck.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ReadinessCheck.java index b3c85bd..eadbb48 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/ReadinessCheck.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ReadinessCheck.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +package org.apache.hadoop.yarn.service.api.records; import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; @@ -27,6 +27,8 @@ 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 @@ -34,7 +36,8 @@ * 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 application 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 { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Resource.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Resource.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Resource.java index 314dfbb..bda79c9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/resource/Resource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Resource.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.slider.api.resource; +package org.apache.hadoop.yarn.service.api.records; import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; @@ -23,6 +23,8 @@ import java.util.Objects; 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.) @@ -32,7 +34,8 @@ * of profile OR cpu & memory are exepected. 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 application. The resource specified at the app (or global) level can be overriden at the component level. Only one of profile OR cpu & memory are exepected. 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 { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceCLI.java index 5574ebd..c7421ff 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceCLI.java @@ -20,8 +20,9 @@ import org.apache.commons.lang.StringUtils; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.slider.api.resource.Application; +import org.apache.hadoop.yarn.service.api.records.Application; import org.apache.hadoop.yarn.service.client.params.ClientArgs; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,7 +33,7 @@ LoggerFactory.getLogger(ServiceClient.class); protected ServiceClient client; - public int exec(ClientArgs args) throws Throwable { + int exec(ClientArgs args) throws Throwable { if (StringUtils.isEmpty(args.getAction())) { System.out.println(args.usage()); return -1; @@ -55,7 +56,7 @@ public int exec(ClientArgs args) throws Throwable { client.actionFlexByCLI(args); break; case ACTION_STOP: - client.actionStop(args.getClusterName()); + client.actionStop(args.getClusterName(), false); break; case ACTION_DESTROY: // Destroy can happen only if app is already stopped client.actionDestroy(args.getClusterName()); @@ -90,7 +91,12 @@ protected void createServiceClient() { public static void main(String[] args) throws Throwable { ClientArgs clientArgs = new ClientArgs(args); - clientArgs.parse(); + try { + clientArgs.parse(); + } catch (BadCommandArgumentsException 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-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java index 0ed4860..1049698 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java @@ -22,6 +22,8 @@ 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; @@ -40,6 +42,7 @@ 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.LocalResource; @@ -58,34 +61,32 @@ 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.Application; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.client.params.AbstractClusterBuildingActionArgs; 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.SliderKeys; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; +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.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.common.params.AbstractClusterBuildingActionArgs; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadClusterStateException; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.exceptions.UsageException; -import org.apache.slider.core.launch.ClasspathConstructor; -import org.apache.slider.core.launch.JavaCommandLineBuilder; -import org.apache.slider.core.registry.SliderRegistryUtils; -import org.apache.slider.core.zk.ZKIntegration; -import org.apache.slider.core.zk.ZookeeperUtils; -import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; +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; @@ -105,17 +106,21 @@ 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.slider.common.Constants.HADOOP_JAAS_DEBUG; -import static org.apache.slider.common.tools.SliderUtils.*; +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.YARN_QUEUE; +import static org.apache.hadoop.yarn.service.utils.SliderUtils.*; +@InterfaceAudience.Public +@InterfaceStability.Unstable public class ServiceClient extends CompositeService - implements SliderExitCodes, SliderKeys { + implements SliderExitCodes, YarnServiceConstants { private static final Logger LOG = LoggerFactory.getLogger(ServiceClient.class); private SliderFileSystem fs; private YarnClient yarnClient; // Avoid looking up applicationId from fs all the time. private Map cachedAppIds = new ConcurrentHashMap<>(); + private Map cachedAMProxies = new ConcurrentHashMap<>(); + private RegistryOperations registryClient; private CuratorFramework curatorClient; private YarnRPC rpc; @@ -293,7 +298,12 @@ private long parseNumberOfContainers(Component component, String newNumber) { ServiceApiUtil.jsonSerDeser .save(fs.getFileSystem(), ServiceApiUtil.getAppJsonPath(fs, appName), persistedApp, true); - ClientAMProtocol proxy = connectToAM(appName); + ClientAMProtocol proxy = getAMProxy(appName); + if (proxy == null) { + String message = appName + " is not running"; + LOG.error(message); + throw new YarnException(message); + } proxy.flexComponents(requestBuilder.build()); for (Map.Entry entry : original.entrySet()) { LOG.info("[COMPONENT {}]: number of containers changed from {} to {}", @@ -303,9 +313,10 @@ private long parseNumberOfContainers(Component component, String newNumber) { return original; } - public int actionStop(String appName) throws YarnException, IOException { + public int actionStop(String appName, boolean waitForAppStopped) + throws YarnException, IOException { validateClusterName(appName); - getAppIdFromPersistedApp(appName); + getAppId(appName); ApplicationId currentAppId = cachedAppIds.get(appName); ApplicationReport report = yarnClient.getApplicationReport(currentAppId); if (terminatedStates.contains(report.getYarnApplicationState())) { @@ -315,17 +326,29 @@ public int actionStop(String appName) throws YarnException, IOException { } LOG.info("Stopping application {}, with appId = {}", appName, currentAppId); try { - // try to stop the app gracefully. - ClientAMProtocol proxy = connectToAM(appName); - StopRequestProto request = StopRequestProto.newBuilder().build(); - proxy.stop(request); - LOG.info("Application " + appName + " is being gracefully stopped..."); + ClientAMProtocol proxy = getAMProxy(appName, report); + cachedAppIds.remove(appName); + cachedAMProxies.remove(appName); + if (proxy != null) { + // try to stop the app gracefully. + StopRequestProto request = StopRequestProto.newBuilder().build(); + proxy.stop(request); + LOG.info("Application " + appName + " is being gracefully stopped..."); + } else { + yarnClient.killApplication(currentAppId, + appName + " is forcefully killed by user!"); + LOG.info("Forcefully kill the application: " + appName); + 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(1000); + Thread.sleep(2000); report = yarnClient.getApplicationReport(currentAppId); if (terminatedStates.contains(report.getYarnApplicationState())) { LOG.info("Application " + appName + " is stopped."); @@ -358,6 +381,7 @@ public int actionDestroy(String appName) throws Exception { FileSystem fileSystem = fs.getFileSystem(); // remove from the appId cache cachedAppIds.remove(appName); + cachedAMProxies.remove(appName); if (fileSystem.exists(appDir)) { if (fileSystem.delete(appDir, true)) { LOG.info("Successfully deleted application dir for " + appName + ": " @@ -370,7 +394,7 @@ public int actionDestroy(String appName) throws Exception { } } deleteZKNode(appName); - String registryPath = SliderRegistryUtils.registryPathForInstance(appName); + String registryPath = ServiceRegistryUtils.registryPathForInstance(appName); try { getRegistryClient().delete(registryPath, true); } catch (IOException e) { @@ -395,7 +419,7 @@ private synchronized RegistryOperations getRegistryClient() private void deleteZKNode(String clusterName) throws Exception { CuratorFramework curatorFramework = getCuratorClient(); String user = RegistryUtils.currentUser(); - String zkPath = ZKIntegration.mkClusterPath(user, clusterName); + String zkPath = ServiceRegistryUtils.mkClusterPath(user, clusterName); if (curatorFramework.checkExists().forPath(zkPath) != null) { curatorFramework.delete().deletingChildrenIfNeeded().forPath(zkPath); LOG.info("Deleted zookeeper path: " + zkPath); @@ -418,7 +442,7 @@ private synchronized CuratorFramework getCuratorClient() if (curatorClient == null) { curatorClient = CuratorFrameworkFactory.builder().connectString(registryQuorum) - .sessionTimeoutMs(10000).retryPolicy(new RetryNTimes(10, 2000)) + .sessionTimeoutMs(10000).retryPolicy(new RetryNTimes(5, 2000)) .build(); curatorClient.start(); } @@ -433,7 +457,7 @@ private int actionHelp(String actionName, CommonArgs args) private void verifyNoLiveAppInRM(String appname, String action) throws IOException, YarnException { Set types = new HashSet<>(1); - types.add(SliderKeys.APP_TYPE); + types.add(YarnServiceConstants.APP_TYPE); Set tags = null; if (appname != null) { tags = Collections.singleton(SliderUtils.createNameTag(appname)); @@ -469,12 +493,13 @@ private ApplicationId submitApp(Application app) appTimeout.put(ApplicationTimeoutType.LIFETIME, app.getLifetime()); submissionContext.setApplicationTimeouts(appTimeout); } - submissionContext.setMaxAppAttempts(conf.getInt(KEY_AM_RESTART_LIMIT, 2)); + 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 hasSliderAMLog4j = + boolean hasAMLog4j = addAMLog4jResource(appName, conf, localResources); // copy jars to hdfs and add to localResources addJarResource(appName, localResources); @@ -487,17 +512,17 @@ private ApplicationId submitApp(Application app) // create AM CLI String cmdStr = - buildCommandLine(appName, conf, appRootDir, hasSliderAMLog4j); + buildCommandLine(appName, conf, appRootDir, hasAMLog4j); submissionContext.setResource(Resource.newInstance(YarnServiceConf - .getLong(KEY_AM_RESOURCE_MEM, DEFAULT_KEY_AM_RESOURCE_MEM, + .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(KEY_YARN_QUEUE, "default"); + queue = conf.get(YARN_QUEUE, "default"); } submissionContext.setQueue(queue); submissionContext.setApplicationName(appName); - submissionContext.setApplicationType(SliderKeys.APP_TYPE); + submissionContext.setApplicationType(YarnServiceConstants.APP_TYPE); Set appTags = AbstractClientProvider.createApplicationTags(appName, null, null); if (!appTags.isEmpty()) { @@ -531,7 +556,7 @@ private String buildCommandLine(String appName, Configuration conf, //TODO CLI.setJVMHeap //TODO CLI.addJVMOPTS if (hasSliderAMLog4j) { - CLI.sysprop(SYSPROP_LOG4J_CONFIGURATION, LOG4J_SERVER_PROP_FILENAME); + CLI.sysprop(SYSPROP_LOG4J_CONFIGURATION, YARN_SERVICE_LOG4J_FILENAME); CLI.sysprop(SYSPROP_LOG_DIR, ApplicationConstants.LOG_DIR_EXPANSION_VAR); } CLI.add(ServiceMaster.class.getCanonicalName()); @@ -553,15 +578,15 @@ private String buildCommandLine(String appName, Configuration conf, private Map addAMEnv(Configuration conf) throws IOException { Map env = new HashMap<>(); ClasspathConstructor classpath = - buildClasspath(SliderKeys.SUBMITTED_CONF_DIR, "lib", fs, getConfig() + 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); + String jaas = System.getenv("HADOOP_JAAS_DEBUG"); if (jaas != null) { - env.put(HADOOP_JAAS_DEBUG, jaas); + env.put("HADOOP_JAAS_DEBUG", jaas); } if (!UserGroupInformation.isSecurityEnabled()) { String userName = UserGroupInformation.getCurrentUser().getUserName(); @@ -579,11 +604,11 @@ protected Path addJarResource(String appName, throws IOException, SliderException { Path libPath = fs.buildClusterDirPath(appName); ProviderUtils - .addProviderJar(localResources, ServiceMaster.class, SLIDER_JAR, fs, + .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() + ": " + LOG.info("Loading lib tar from " + fs.getFileSystem().getScheme() + ":/" + dependencyLibTarGzip); SliderUtils.putAmTarGzipAndUpdate(localResources, fs); } else { @@ -599,27 +624,29 @@ protected Path addJarResource(String appName, private boolean addAMLog4jResource(String appName, Configuration conf, Map localResources) throws IOException, BadClusterStateException { - boolean hasSliderAMLog4j = false; + boolean hasAMLog4j = false; String hadoopConfDir = System.getenv(ApplicationConstants.Environment.HADOOP_CONF_DIR.name()); if (hadoopConfDir != null) { File localFile = - new File(hadoopConfDir, SliderKeys.LOG4J_SERVER_PROP_FILENAME); + new File(hadoopConfDir, YarnServiceConstants.YARN_SERVICE_LOG4J_FILENAME); if (localFile.exists()) { Path localFilePath = createLocalPath(localFile); Path appDirPath = fs.buildClusterDirPath(appName); Path remoteConfPath = - new Path(appDirPath, SliderKeys.SUBMITTED_CONF_DIR); + new Path(appDirPath, YarnServiceConstants.SUBMITTED_CONF_DIR); Path remoteFilePath = - new Path(remoteConfPath, SliderKeys.LOG4J_SERVER_PROP_FILENAME); + new Path(remoteConfPath, YarnServiceConstants.YARN_SERVICE_LOG4J_FILENAME); copy(conf, localFilePath, remoteFilePath); LocalResource localResource = fs.createAmResource(remoteConfPath, LocalResourceType.FILE); localResources.put(localFilePath.getName(), localResource); - hasSliderAMLog4j = true; + hasAMLog4j = true; + } else { + LOG.warn("AM log4j property file doesn't exist: " + localFile); } } - return hasSliderAMLog4j; + return hasAMLog4j; } public int actionStart(String appName) throws YarnException, IOException { @@ -674,22 +701,22 @@ private void addKeytabResourceIfSecure(SliderFileSystem fileSystem, return; } String keytabPreInstalledOnHost = - conf.get(SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH); + conf.get(YarnServiceConf.KEY_AM_KEYTAB_LOCAL_PATH); if (StringUtils.isEmpty(keytabPreInstalledOnHost)) { String amKeytabName = - conf.get(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME); - String keytabDir = conf.get(SliderXmlConfKeys.KEY_HDFS_KEYTAB_DIR); + conf.get(YarnServiceConf.KEY_AM_LOGIN_KEYTAB_NAME); + String keytabDir = conf.get(YarnServiceConf.KEY_HDFS_KEYTAB_DIR); Path keytabPath = fileSystem.buildKeytabPath(keytabDir, amKeytabName, appName); if (fileSystem.getFileSystem().exists(keytabPath)) { LocalResource keytabRes = fileSystem.createAmResource(keytabPath, LocalResourceType.FILE); localResource - .put(SliderKeys.KEYTAB_DIR + "/" + amKeytabName, keytabRes); + .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(KEY_AM_LOGIN_KEYTAB_REQUIRED, false)) { + if (conf.getBoolean(YarnServiceConf.KEY_AM_LOGIN_KEYTAB_REQUIRED, false)) { throw new BadConfigException("No keytab file was found at %s.", keytabPath); } else { @@ -704,7 +731,7 @@ private void addKeytabResourceIfSecure(SliderFileSystem fileSystem, public String updateLifetime(String appName, long lifetime) throws YarnException, IOException { - getAppIdFromPersistedApp(appName); + getAppId(appName); ApplicationId currentAppId = cachedAppIds.get(appName); ApplicationReport report = yarnClient.getApplicationReport(currentAppId); if (report == null) { @@ -729,11 +756,25 @@ public String updateLifetime(String appName, long lifetime) public Application getStatus(String appName) throws IOException, YarnException { - ClientAMProtocol proxy = connectToAM(appName); - GetStatusResponseProto response = - proxy.getStatus(GetStatusRequestProto.newBuilder().build()); - return ServiceApiUtil.jsonSerDeser.fromJson(response.getStatus()); - + validateClusterName(appName); + ApplicationId currentAppId = getAppId(appName); + ApplicationReport appReport = yarnClient.getApplicationReport(currentAppId); + ClientAMProtocol amProxy = getAMProxy(appName, appReport); + Application appSpec; + if (amProxy != null) { + GetStatusResponseProto response = + amProxy.getStatus(GetStatusRequestProto.newBuilder().build()); + appSpec = ServiceApiUtil.jsonSerDeser.fromJson(response.getStatus()); + } else { + appSpec = new Application(); + appSpec.setName(appName); + } + ApplicationTimeout lifetime = + appReport.getApplicationTimeouts().get(ApplicationTimeoutType.LIFETIME); + if (lifetime != null) { + appSpec.setLifetime(lifetime.getRemainingTime()); + } + return appSpec; } public YarnClient getYarnClient() { @@ -760,71 +801,61 @@ public int actionDependency(ActionDependencyArgs args) String[] libDirs = SliderUtils.getLibDirs(); if (libDirs.length > 0) { File tempLibTarGzipFile = File.createTempFile( - SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_NAME + "_", - SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_EXT); + YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_NAME + "_", + YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_EXT); // copy all jars tarGzipFolder(libDirs, tempLibTarGzipFile, createJarFilter()); - LOG.info("Uploading dependency for AM (version {}) from {} to {}", - VersionInfo.getBuildVersion(), tempLibTarGzipFile.toURI(), - dependencyLibTarGzip.toUri()); + LOG.info("Version Info: " + VersionInfo.getBuildVersion()); fs.copyLocalFileToHdfs(tempLibTarGzipFile, dependencyLibTarGzip, - new FsPermission(SliderKeys.SLIDER_DEPENDENCY_DIR_PERMISSIONS)); + new FsPermission(YarnServiceConstants.DEPENDENCY_DIR_PERMISSIONS)); return EXIT_SUCCESS; } else { return EXIT_FALSE; } } - protected ClientAMProtocol connectToAM(String appName) + // Get AMProxy with the appReport provided + protected ClientAMProtocol getAMProxy(String appName, ApplicationReport report) + throws IOException { + if (!cachedAMProxies.containsKey(appName) && !StringUtils + .isEmpty(report.getHost())) { + insertAMProxy(appName, report.getHost(), report.getRpcPort()); + } + return cachedAMProxies.get(appName); + } + + // Get AMProxy without appReport provided - it'll getAppReport from RM + protected ClientAMProtocol getAMProxy(String appName) throws IOException, YarnException { - ApplicationId currentAppId = getAppIdFromPersistedApp(appName); - // Wait until app becomes running. - long startTime = System.currentTimeMillis(); - int pollCount = 0; - ApplicationReport appReport = null; - while (true) { - appReport = yarnClient.getApplicationReport(currentAppId); - YarnApplicationState state = appReport.getYarnApplicationState(); - if (state == RUNNING) { - break; - } - if (terminatedStates.contains(state)) { - throw new YarnException( - "Failed to getStatus " + currentAppId + ": " + appReport - .getDiagnostics()); - } - long elapsedMillis = System.currentTimeMillis() - startTime; - // if over 5 min, quit - if (elapsedMillis >= 300000) { - throw new YarnException( - "Timed out while waiting for application " + currentAppId - + " to be running"); - } + ApplicationId currentAppId = getAppId(appName); - if (++pollCount % 10 == 0) { - LOG.info( - "Waiting for application {} to be running, current state is {}", - currentAppId, state); - } - try { - Thread.sleep(3000); - } catch (InterruptedException ie) { - String msg = - "Interrupted while waiting for application " + currentAppId - + " to be running."; - throw new YarnException(msg, ie); + if (cachedAMProxies.containsKey(appName)) { + return cachedAMProxies.get(appName); + } else { + ApplicationReport appReport = + yarnClient.getApplicationReport(currentAppId); + String host = appReport.getHost(); + int port = appReport.getRpcPort(); + if (!StringUtils.isEmpty(host)) { + return insertAMProxy(appName, host, port); } + return null; } + } - // Make the connection - InetSocketAddress address = NetUtils - .createSocketAddrForHost(appReport.getHost(), appReport.getRpcPort()); - return ClientAMProxy.createProxy(getConfig(), ClientAMProtocol.class, + private ClientAMProtocol insertAMProxy(String appName, String host, int port) + throws IOException { + InetSocketAddress address = + NetUtils.createSocketAddrForHost(host, port); + ClientAMProtocol amProxy = + ClientAMProxy.createProxy(getConfig(), ClientAMProtocol.class, UserGroupInformation.getCurrentUser(), rpc, address); + cachedAMProxies.put(appName, amProxy); + return amProxy; } - private synchronized ApplicationId getAppIdFromPersistedApp(String appName) + private synchronized ApplicationId getAppId(String appName) throws IOException, YarnException { if (cachedAppIds.containsKey(appName)) { return cachedAppIds.get(appName); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractActionArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractActionArgs.java index 05c6501..ea3bb0a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractActionArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractActionArgs.java @@ -20,10 +20,9 @@ import com.beust.jcommander.Parameter; import org.apache.hadoop.fs.Path; -import org.apache.slider.common.params.PathArgumentConverter; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.ErrorStrings; -import org.apache.slider.core.exceptions.UsageException; +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; @@ -49,7 +48,7 @@ protected AbstractActionArgs() { public String filesystemBinding; @Parameter(names = {ARG_BASE_PATH}, - description = "Slider base path on the filesystem", + description = "Service base path on the filesystem", converter = PathArgumentConverter.class) public Path basePath; @@ -129,7 +128,7 @@ public void validate() throws BadCommandArgumentsException, UsageException { if (minArgs > actionArgSize) { throw new BadCommandArgumentsException( ErrorStrings.ERROR_NOT_ENOUGH_ARGUMENTS + getActionName() + - " Expected minimum " + minArgs + " but got " + actionArgSize); + ", Expected minimum " + minArgs + " but got " + actionArgSize); } int maxArgs = getMaxParams(); if (maxArgs == -1) { @@ -146,7 +145,7 @@ public void validate() throws BadCommandArgumentsException, UsageException { StringBuilder buf = new StringBuilder(message); for (String actionArg : parameters) { log.error("[{}] \"{}\"", index++, actionArg); - buf.append(" \"" + actionArg + "\" "); + buf.append(" \"").append(actionArg).append("\" "); } throw new BadCommandArgumentsException(buf.toString()); } @@ -156,25 +155,4 @@ public void validate() throws BadCommandArgumentsException, UsageException { public String toString() { return super.toString() + ": " + getActionName(); } - - /** - * Override point: - * Flag to indicate that core hadoop API services are needed (HDFS, YARN, etc) - * —and that validation of the client state should take place. - * - * @return a flag to indicate that the core hadoop services will be needed. - */ - public boolean getHadoopServicesRequired() { - return true; - } - - /** - * Flag to disable secure login. - * This MUST only be set if the action is bypassing security or setting - * it itself - * @return true if login at slider client init time is to be skipped - */ - public boolean disableSecureLogin() { - return false; - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractArgsDelegate.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractArgsDelegate.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractArgsDelegate.java index ec88ca1..457e357 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractArgsDelegate.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractArgsDelegate.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.params; +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; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractClusterBuildingActionArgs.java similarity index 61% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractClusterBuildingActionArgs.java index 57c27e7..017286f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/AbstractClusterBuildingActionArgs.java @@ -16,14 +16,12 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameter; import com.beust.jcommander.ParametersDelegate; import com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; -import org.apache.hadoop.yarn.service.client.params.ComponentArgsDelegate; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; import java.io.File; import java.util.List; @@ -57,35 +55,4 @@ public File getAppDef() { @ParametersDelegate public OptionArgsDelegate optionsDelegate = new OptionArgsDelegate(); - - - public Map getOptionsMap() throws - BadCommandArgumentsException { - return optionsDelegate.getOptionsMap(); - } - - /** - * Get the role heap mapping (may be empty, but never null). - * @return role heap mapping - * @throws BadCommandArgumentsException parse problem - */ - public Map> getCompOptionMap() throws - BadCommandArgumentsException { - return optionsDelegate.getCompOptionMap(); - } - - @VisibleForTesting - public List getComponentTuples() { - return componentDelegate.getComponentTuples(); - } - - /** - * Get the role mapping (may be empty, but never null). - * @return role 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-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionBuildArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionBuildArgs.java index 28381cf..c2ff545 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionBuildArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionBuildArgs.java @@ -18,8 +18,6 @@ package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameters; -import org.apache.hadoop.yarn.service.client.params.SliderActions; -import org.apache.slider.common.params.AbstractClusterBuildingActionArgs; @Parameters(commandNames = { SliderActions.ACTION_BUILD}, commandDescription = SliderActions.DESCRIBE_ACTION_BUILD) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionClientArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionClientArgs.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionClientArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionClientArgs.java index 8dfde36..0097b4e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionClientArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionClientArgs.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameters; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionCreateArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionCreateArgs.java index 35cef5a..eecffb6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionCreateArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionCreateArgs.java @@ -19,8 +19,6 @@ package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameters; -import org.apache.hadoop.yarn.service.client.params.SliderActions; -import org.apache.slider.common.params.AbstractClusterBuildingActionArgs; @Parameters(commandNames = { SliderActions.ACTION_CREATE}, commandDescription = SliderActions.DESCRIBE_ACTION_CREATE) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionDependencyArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionDependencyArgs.java index b41b2af..51e07c9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionDependencyArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionDependencyArgs.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.yarn.service.client.params; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.UsageException; +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; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionExistsArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionExistsArgs.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionExistsArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionExistsArgs.java index b075ce0..ba3c5a9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionExistsArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionExistsArgs.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameters; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionFlexArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionFlexArgs.java index fcbb803..b7acf58 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionFlexArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionFlexArgs.java @@ -20,7 +20,7 @@ import com.beust.jcommander.Parameters; import com.beust.jcommander.ParametersDelegate; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; import java.util.List; import java.util.Map; @@ -47,9 +47,4 @@ public String getActionName() { BadCommandArgumentsException { return componentDelegate.getComponentMap(); } - - public List getComponentTuples() { - return componentDelegate.getComponentTuples(); - } - } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionFreezeArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionFreezeArgs.java similarity index 91% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionFreezeArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionFreezeArgs.java index b08e1cf..aecf0eb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionFreezeArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionFreezeArgs.java @@ -16,13 +16,11 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameters; import com.beust.jcommander.ParametersDelegate; -import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; -import org.apache.hadoop.yarn.service.client.params.SliderActions; @Parameters(commandNames = { SliderActions.ACTION_STOP }, commandDescription = SliderActions.DESCRIBE_ACTION_FREEZE) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionHelpArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionHelpArgs.java similarity index 87% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionHelpArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionHelpArgs.java index fc6eb4f..51aa88a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionHelpArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionHelpArgs.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameters; import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; @@ -41,13 +41,4 @@ public String getActionName() { public int getMinParams() { return 0; } - - /** - * This action does not need hadoop services - * @return false - */ - @Override - public boolean getHadoopServicesRequired() { - return false; - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKDiagArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionKDiagArgs.java similarity index 82% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKDiagArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionKDiagArgs.java index be370bb..061121e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKDiagArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionKDiagArgs.java @@ -16,15 +16,13 @@ * limitations under the License. */ -package org.apache.slider.common.params; +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 org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.UsageException; +import org.apache.hadoop.yarn.service.utils.SliderUtils; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.UsageException; import java.io.File; import java.util.ArrayList; @@ -66,16 +64,6 @@ public int getMinParams() { } @Override - public boolean getHadoopServicesRequired() { - return false; - } - - @Override - public boolean disableSecureLogin() { - return true; - } - - @Override public void validate() throws BadCommandArgumentsException, UsageException { super.validate(); if (keytab != null && SliderUtils.isUnset(principal)) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKeytabArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionKeytabArgs.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKeytabArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionKeytabArgs.java index 7a46c66..7e51457 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKeytabArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionKeytabArgs.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameters; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionListArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionListArgs.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionListArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionListArgs.java index 51bde7b..005c172 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionListArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionListArgs.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import java.util.HashSet; import java.util.Set; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionRegistryArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionRegistryArgs.java similarity index 91% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionRegistryArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionRegistryArgs.java index fb76451..c2866cf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionRegistryArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionRegistryArgs.java @@ -16,17 +16,14 @@ * limitations under the License. */ -package org.apache.slider.common.params; +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.Arguments; -import org.apache.hadoop.yarn.service.client.params.SliderActions; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.UsageException; -import org.apache.slider.core.registry.docstore.ConfigFormat; +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; @@ -128,7 +125,7 @@ public int getMinParams() { @Parameter(names = {ARG_SERVICETYPE}, description = "optional service type") - public String serviceType = SliderKeys.APP_TYPE; + public String serviceType = YarnServiceConstants.APP_TYPE; @Parameter(names = {ARG_VERBOSE}, description = "verbose output") diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResolveArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionResolveArgs.java similarity index 92% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResolveArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionResolveArgs.java index a953bc7..65f0472 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResolveArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionResolveArgs.java @@ -16,15 +16,13 @@ * limitations under the License. */ -package org.apache.slider.common.params; +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.client.params.AbstractActionArgs; -import org.apache.hadoop.yarn.service.client.params.SliderActions; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.UsageException; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.UsageException; import java.io.File; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResourceArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionResourceArgs.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResourceArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionResourceArgs.java index 6d60ca7..b03dc92 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResourceArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionResourceArgs.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameters; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionStatusArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionStatusArgs.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionStatusArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionStatusArgs.java index 5285f7b..622e77d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionStatusArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionStatusArgs.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameters; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionThawArgs.java similarity index 91% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionThawArgs.java index e8bdcad..2b90479 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionThawArgs.java @@ -16,13 +16,11 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameters; import com.beust.jcommander.ParametersDelegate; -import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; -import org.apache.hadoop.yarn.service.client.params.SliderActions; import java.io.File; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionTokensArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionTokensArgs.java similarity index 88% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionTokensArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionTokensArgs.java index f1f0125..cf48513 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionTokensArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionTokensArgs.java @@ -16,14 +16,12 @@ * limitations under the License. */ -package org.apache.slider.common.params; +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 org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.UsageException; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.UsageException; import java.io.File; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpdateArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionUpdateArgs.java similarity index 91% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpdateArgs.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionUpdateArgs.java index 830e4ee..e310f45 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpdateArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ActionUpdateArgs.java @@ -16,10 +16,9 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameters; -import org.apache.hadoop.yarn.service.client.params.SliderActions; @Parameters(commandNames = { SliderActions.ACTION_UPDATE}, commandDescription = SliderActions.DESCRIBE_ACTION_UPDATE) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ArgOps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ArgOps.java index f7b7349..00151f4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ArgOps.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ArgOps.java @@ -20,8 +20,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.ErrorStrings; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.ErrorStrings; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/Arguments.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/Arguments.java index d8d8ab4..204149b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/Arguments.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/Arguments.java @@ -26,59 +26,38 @@ * so it is easier to see what arguments are there */ public interface Arguments { - String ARG_ADDON = "--addon"; - String ARG_ALL = "--all"; - String ARG_ALIAS = "--alias"; - String ARG_APPLICATION = "--application"; + String ARG_APPDEF = "--appdef"; - String ARG_APP_HOME = "--apphome"; String ARG_BASE_PATH = "--basepath"; - String ARG_CLIENT = "--client"; - String ARG_CONFDIR = "--appconf"; String ARG_COMPONENT = "--component"; - String ARG_COUNT = "--count"; 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_CREDENTIALS = "--credentials"; String ARG_DEBUG = "--debug"; String ARG_DEFINE = "-D"; String ARG_DELETE = "--delete"; String ARG_DEST = "--dest"; String ARG_DESTDIR = "--destdir"; - String ARG_DESTFILE = "--destfile"; - String ARG_EXITCODE = "--exitcode"; - String ARG_FAIL = "--fail"; - /** - filesystem-uri: {@value} - */ String ARG_FILESYSTEM = "--fs"; String ARG_FILESYSTEM_LONG = "--filesystem"; String ARG_FOLDER = "--folder"; String ARG_FORCE = "--force"; String ARG_FORMAT = "--format"; - String ARG_GETCERTSTORE = "--getcertstore"; String ARG_GETCONF = "--getconf"; String ARG_GETEXP = "--getexp"; String ARG_GETFILES = "--getfiles"; - String ARG_HEALTHY= "--healthy"; String ARG_HELP = "--help"; - String ARG_HOSTNAME = "--hostname"; - String ARG_ID = "--id"; String ARG_IMAGE = "--image"; String ARG_INSTALL = "--install"; String ARG_INTERNAL = "--internal"; String ARG_KEYLEN = "--keylen"; String ARG_KEYTAB = "--keytab"; - String ARG_KEYSTORE = "--keystore"; String ARG_KEYTABINSTALL = ARG_INSTALL; String ARG_KEYTABDELETE = ARG_DELETE; String ARG_KEYTABLIST = "--list"; - String ARG_LABEL = "--label"; - String ARG_LEVEL = "--level"; String ARG_LIST = "--list"; String ARG_LISTCONF = "--listconf"; String ARG_LISTEXP = "--listexp"; @@ -87,8 +66,6 @@ String ARG_MANAGER = "--manager"; String ARG_MANAGER_SHORT = "--m"; String ARG_MESSAGE = "--message"; - String ARG_METAINFO = "--metainfo"; - String ARG_METAINFO_JSON = "--metainfojson"; String ARG_NAME = "--name"; String ARG_OPTION = "--option"; String ARG_OPTION_SHORT = "-O"; @@ -96,35 +73,23 @@ String ARG_OUTPUT_SHORT = "-o"; String ARG_OVERWRITE = "--overwrite"; String ARG_PACKAGE = "--package"; - String ARG_PASSWORD = "--password"; String ARG_PATH = "--path"; - String ARG_PKGDELETE = ARG_DELETE; - String ARG_PKGINSTANCES = "--instances"; - String ARG_PKGLIST = ARG_LIST; String ARG_PRINCIPAL = "--principal"; - String ARG_PROVIDER = "--provider"; String ARG_QUEUE = "--queue"; String ARG_LIFETIME = "--lifetime"; - String ARG_REPLACE_PKG = "--replacepkg"; String ARG_RESOURCE = "--resource"; String ARG_RESOURCE_MANAGER = "--rm"; String ARG_SECURE = "--secure"; String ARG_SERVICETYPE = "--servicetype"; String ARG_SERVICES = "--services"; - String ARG_SLIDER = "--slider"; String ARG_SOURCE = "--source"; String ARG_STATE = "--state"; String ARG_SYSPROP = "-S"; - String ARG_TRUSTSTORE = "--truststore"; String ARG_USER = "--user"; String ARG_UPLOAD = "--upload"; String ARG_VERBOSE = "--verbose"; String ARG_VERSION = "--version"; String ARG_WAIT = "--wait"; - String ARG_YARN = "--yarn"; - String ARG_ZKHOSTS = "--zkhosts"; - String ARG_ZKPATH = "--zkpath"; - String ARG_ZKPORT = "--zkport"; /* STOP: DO NOT ADD YOUR ARGUMENTS HERE. GO BACK AND INSERT THEM IN THE RIGHT PLACE IN THE LIST diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ClientArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ClientArgs.java index e85db58..7b957fa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ClientArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ClientArgs.java @@ -20,78 +20,36 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; -import org.apache.slider.common.params.AbstractClusterBuildingActionArgs; -import org.apache.slider.common.params.ActionAMSuicideArgs; -import org.apache.slider.common.params.ActionClientArgs; -import org.apache.slider.common.params.ActionDiagnosticArgs; -import org.apache.slider.common.params.ActionExistsArgs; -import org.apache.slider.common.params.ActionFreezeArgs; -import org.apache.slider.common.params.ActionHelpArgs; -import org.apache.slider.common.params.ActionKDiagArgs; -import org.apache.slider.common.params.ActionKeytabArgs; -import org.apache.slider.common.params.ActionKillContainerArgs; -import org.apache.slider.common.params.ActionListArgs; -import org.apache.slider.common.params.ActionLookupArgs; -import org.apache.slider.common.params.ActionNodesArgs; -import org.apache.slider.common.params.ActionRegistryArgs; -import org.apache.slider.common.params.ActionResolveArgs; -import org.apache.slider.common.params.ActionResourceArgs; -import org.apache.slider.common.params.ActionStatusArgs; -import org.apache.slider.common.params.ActionThawArgs; -import org.apache.slider.common.params.ActionTokensArgs; -import org.apache.slider.common.params.ActionUpdateArgs; -import org.apache.slider.common.params.ActionUpgradeArgs; -import org.apache.slider.common.params.ActionVersionArgs; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.ErrorStrings; -import org.apache.slider.core.exceptions.SliderException; +import org.apache.hadoop.yarn.service.conf.YarnServiceConf; +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; /** - * Slider Client CLI Args + * Client CLI Args */ public class ClientArgs extends CommonArgs { - /* - - All the arguments for specific actions - - */ - /** - * This is not bonded to jcommander, it is set up - * after the construction to point to the relevant - * entry - * - * KEEP IN ALPHABETICAL ORDER - */ - private AbstractClusterBuildingActionArgs buildingActionArgs; - // ========================================================= // Keep all of these in alphabetical order. Thanks. // ========================================================= - private final ActionAMSuicideArgs actionAMSuicideArgs = new ActionAMSuicideArgs(); 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 ActionDiagnosticArgs actionDiagnosticArgs = new ActionDiagnosticArgs(); 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 ActionKDiagArgs actionKDiagArgs = new ActionKDiagArgs(); private final ActionKeytabArgs actionKeytabArgs = new ActionKeytabArgs(); - private final ActionKillContainerArgs actionKillContainerArgs = - new ActionKillContainerArgs(); private final ActionListArgs actionListArgs = new ActionListArgs(); - private final ActionLookupArgs actionLookupArgs = new ActionLookupArgs(); - private final ActionNodesArgs actionNodesArgs = new ActionNodesArgs(); private final ActionRegistryArgs actionRegistryArgs = new ActionRegistryArgs(); private final ActionResolveArgs actionResolveArgs = new ActionResolveArgs(); private final ActionResourceArgs actionResourceArgs = new ActionResourceArgs(); @@ -99,8 +57,6 @@ private final ActionThawArgs actionThawArgs = new ActionThawArgs(); private final ActionTokensArgs actionTokenArgs = new ActionTokensArgs(); private final ActionUpdateArgs actionUpdateArgs = new ActionUpdateArgs(); - private final ActionUpgradeArgs actionUpgradeArgs = new ActionUpgradeArgs(); - private final ActionVersionArgs actionVersionArgs = new ActionVersionArgs(); public ClientArgs(String[] args) { super(args); @@ -114,32 +70,15 @@ public ClientArgs(Collection args) { protected void addActionArguments() { addActions( - actionAMSuicideArgs, actionBuildArgs, - actionClientArgs, actionCreateArgs, actionDependencyArgs, actionDestroyArgs, - actionDiagnosticArgs, - actionExistsArgs, actionFlexArgs, actionFreezeArgs, actionHelpArgs, - actionKDiagArgs, - actionKeytabArgs, - actionKillContainerArgs, - actionListArgs, - actionLookupArgs, - actionNodesArgs, - actionRegistryArgs, - actionResolveArgs, - actionResourceArgs, actionStatusArgs, - actionThawArgs, - actionTokenArgs, - actionUpdateArgs, - actionUpgradeArgs, - actionVersionArgs + actionThawArgs ); } @@ -154,43 +93,20 @@ public void applyDefinitions(Configuration conf) throws } if (getBasePath() != null) { log.debug("Setting basePath to {}", getBasePath()); - conf.set(SliderXmlConfKeys.KEY_SLIDER_BASE_PATH, + conf.set(YarnServiceConf.YARN_SERVICE_BASE_PATH, getBasePath().toString()); } } - public ActionDiagnosticArgs getActionDiagnosticArgs() { - return actionDiagnosticArgs; - } - - public AbstractClusterBuildingActionArgs getBuildingActionArgs() { - return buildingActionArgs; - } - - public ActionAMSuicideArgs getActionAMSuicideArgs() { - return actionAMSuicideArgs; - } public ActionBuildArgs getActionBuildArgs() { return actionBuildArgs; } - public ActionClientArgs getActionClientArgs() { return actionClientArgs; } - - public ActionKDiagArgs getActionKDiagArgs() { - return actionKDiagArgs; - } - - public ActionKeytabArgs getActionKeytabArgs() { return actionKeytabArgs; } - public ActionUpdateArgs getActionUpdateArgs() { return actionUpdateArgs; } - public ActionUpgradeArgs getActionUpgradeArgs() { - return actionUpgradeArgs; - } - public ActionCreateArgs getActionCreateArgs() { return actionCreateArgs; } @@ -215,21 +131,10 @@ public ActionFreezeArgs getActionFreezeArgs() { return actionFreezeArgs; } - public ActionKillContainerArgs getActionKillContainerArgs() { - return actionKillContainerArgs; - } - public ActionListArgs getActionListArgs() { return actionListArgs; } - public ActionNodesArgs getActionNodesArgs() { - return actionNodesArgs; - } - - public ActionLookupArgs getActionLookupArgs() { - return actionLookupArgs; - } public ActionRegistryArgs getActionRegistryArgs() { return actionRegistryArgs; @@ -268,14 +173,10 @@ public void applyAction() throws SliderException { switch (action) { case ACTION_BUILD: bindCoreAction(actionBuildArgs); - //its a builder, so set those actions too - buildingActionArgs = actionBuildArgs; break; case ACTION_CREATE: bindCoreAction(actionCreateArgs); - //its a builder, so set those actions too - buildingActionArgs = actionCreateArgs; break; case ACTION_STOP: @@ -286,14 +187,6 @@ public void applyAction() throws SliderException { bindCoreAction(actionThawArgs); break; - case ACTION_AM_SUICIDE: - bindCoreAction(actionAMSuicideArgs); - break; - - case ACTION_CLIENT: - bindCoreAction(actionClientArgs); - break; - case ACTION_DEPENDENCY: bindCoreAction(actionDependencyArgs); break; @@ -302,10 +195,6 @@ public void applyAction() throws SliderException { bindCoreAction(actionDestroyArgs); break; - case ACTION_DIAGNOSTICS: - bindCoreAction(actionDiagnosticArgs); - break; - case ACTION_EXISTS: bindCoreAction(actionExistsArgs); break; @@ -326,22 +215,10 @@ public void applyAction() throws SliderException { bindCoreAction(actionKeytabArgs); break; - case ACTION_KILL_CONTAINER: - bindCoreAction(actionKillContainerArgs); - break; - case ACTION_LIST: bindCoreAction(actionListArgs); break; - case ACTION_LOOKUP: - bindCoreAction(actionLookupArgs); - break; - - case ACTION_NODES: - bindCoreAction(actionNodesArgs); - break; - case ACTION_REGISTRY: bindCoreAction(actionRegistryArgs); break; @@ -366,14 +243,6 @@ public void applyAction() throws SliderException { bindCoreAction(actionUpdateArgs); break; - case ACTION_UPGRADE: - bindCoreAction(actionUpgradeArgs); - break; - - case ACTION_VERSION: - bindCoreAction(actionVersionArgs); - break; - default: throw new BadCommandArgumentsException(ErrorStrings.ERROR_UNKNOWN_ACTION + " " + action); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/CommonArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/CommonArgs.java index 3160512..d44412a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/CommonArgs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/CommonArgs.java @@ -25,11 +25,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.ErrorStrings; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.exceptions.UsageException; +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; @@ -112,7 +112,7 @@ public static String usage(CommonArgs serviceArgs, String commandOfInterest) { 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: slider COMMAND [options]\n"); + 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%-" @@ -123,7 +123,7 @@ public static String usage(CommonArgs serviceArgs, String commandOfInterest) { .append("Most commands print help when invoked without parameters or with --help"); result = helperMessage.toString(); } else { - helperMessage.append("\nUsage: slider ").append(commandOfInterest); + helperMessage.append("\nUsage: service ").append(commandOfInterest); helperMessage.append(serviceArgs.coreAction.getMinParams() > 0 ? " " : ""); helperMessage.append("\n"); for (ParameterDescription paramDesc : serviceArgs.commander.getCommands() @@ -224,14 +224,6 @@ protected void bindCoreAction(AbstractActionArgs action) { } /** - * Get the core action -type depends on the action - * @return the action class - */ - public AbstractActionArgs getCoreAction() { - return coreAction; - } - - /** * Validate the arguments against the action requested */ public void validate() throws BadCommandArgumentsException, UsageException { @@ -244,7 +236,8 @@ public void validate() throws BadCommandArgumentsException, UsageException { coreAction.validate(); } catch (BadCommandArgumentsException e) { String badArgMsgBuilder = - e.toString() + "\n" + usage(this, coreAction.getActionName()); + e.getMessage() + System.lineSeparator() + usage(this, + coreAction.getActionName()); throw new BadCommandArgumentsException(badArgMsgBuilder); } } @@ -286,9 +279,4 @@ public String getManager() { public String getAction() { return commander.getParsedCommand(); } - - public List getActionArgs() { - return coreAction.parameters; - } - } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ComponentArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ComponentArgsDelegate.java index 0bdf58e..b6cd0a1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ComponentArgsDelegate.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/ComponentArgsDelegate.java @@ -19,9 +19,7 @@ package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameter; -import org.apache.slider.common.params.AbstractArgsDelegate; -import org.apache.slider.common.params.DontSplitArguments; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; import java.util.ArrayList; import java.util.List; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/DontSplitArguments.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/DontSplitArguments.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/DontSplitArguments.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/DontSplitArguments.java index 0344305..85de615 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/DontSplitArguments.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/DontSplitArguments.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.converters.IParameterSplitter; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsAccessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/LaunchArgsAccessor.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsAccessor.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/LaunchArgsAccessor.java index 7524053..bf194b6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsAccessor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/LaunchArgsAccessor.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import java.io.File; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/LaunchArgsDelegate.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsDelegate.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/LaunchArgsDelegate.java index bc7e94c..157fb61 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsDelegate.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/LaunchArgsDelegate.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameter; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/OptionArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/OptionArgsDelegate.java similarity index 94% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/OptionArgsDelegate.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/OptionArgsDelegate.java index e63bd12..7972716 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/OptionArgsDelegate.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/OptionArgsDelegate.java @@ -16,10 +16,10 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameter; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; import java.util.ArrayList; import java.util.List; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/PathArgumentConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/PathArgumentConverter.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/PathArgumentConverter.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/PathArgumentConverter.java index ccb526c..040ac64 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/PathArgumentConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/PathArgumentConverter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.converters.BaseConverter; import org.apache.hadoop.fs.Path; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/SliderAMCreateAction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/SliderAMCreateAction.java index 1853229..a446665 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/SliderAMCreateAction.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/SliderAMCreateAction.java @@ -21,10 +21,6 @@ import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameters; import com.beust.jcommander.ParametersDelegate; -import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; -import org.apache.hadoop.yarn.service.client.params.SliderActions; -import org.apache.slider.common.params.LaunchArgsAccessor; -import org.apache.slider.common.params.LaunchArgsDelegate; import java.io.File; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/SliderActions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/SliderActions.java index 3ea6f67..fc3c5a1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/SliderActions.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/SliderActions.java @@ -24,16 +24,13 @@ * are listed here to ensure the names are consistent */ public interface SliderActions { - String ACTION_AM_SUICIDE = "am-suicide"; String ACTION_BUILD = "build"; String ACTION_CLIENT = "client"; String ACTION_CREATE = "create"; - String ACTION_DIAGNOSTICS = "diagnostics"; String ACTION_DEPENDENCY = "dependency"; String ACTION_UPDATE = "update"; String ACTION_UPGRADE = "upgrade"; String ACTION_DESTROY = "destroy"; - String ACTION_ECHO = "echo"; String ACTION_EXISTS = "exists"; String ACTION_FLEX = "flex"; String ACTION_STOP = "stop"; @@ -41,12 +38,8 @@ String ACTION_INSTALL_KEYTAB = "install-keytab"; String ACTION_KDIAG = "kdiag"; String ACTION_KEYTAB = "keytab"; - String ACTION_KILL_CONTAINER = "kill-container"; String ACTION_LIST = "list"; - String ACTION_LOOKUP = "lookup"; - String ACTION_NODES = "nodes"; - String ACTION_PREFLIGHT = "preflight"; - String ACTION_RECONFIGURE = "reconfigure"; + String ACTION_REGISTRY = "registry"; String ACTION_RESOLVE = "resolve"; String ACTION_RESOURCE = "resource"; @@ -54,52 +47,35 @@ String ACTION_START = "start"; String ACTION_TOKENS = "tokens"; - String ACTION_VERSION = "version"; - String DESCRIBE_ACTION_AM_SUICIDE = - "Tell the Slider Application Master to simulate a process failure by terminating itself"; String DESCRIBE_ACTION_BUILD = - "Build a Slider cluster specification, but do not start it"; + "Build a service specification, but do not start it"; String DESCRIBE_ACTION_CREATE = - "Create a live Slider application"; + "Build and start a service, it's equivalent to first invoke build and then start"; String DESCRIBE_ACTION_DEPENDENCY = - "Slider AM and agent dependency (libraries) management"; + "Yarn service framework dependency (libraries) management"; String DESCRIBE_ACTION_UPDATE = - "Update template for a Slider application"; + "Update template for service"; String DESCRIBE_ACTION_UPGRADE = "Rolling upgrade/downgrade the component/containerto a newer/previous version"; String DESCRIBE_ACTION_DESTROY = - "Destroy a stopped Slider application"; + "Destroy a stopped service, service must be stopped first before destroying."; String DESCRIBE_ACTION_EXISTS = "Probe for an application running"; - String DESCRIBE_ACTION_FLEX = "Flex a Slider application"; + String DESCRIBE_ACTION_FLEX = "Flex a service's component by increasing or decreasing the number of containers."; String DESCRIBE_ACTION_FREEZE = - "Stop a running application"; - String DESCRIBE_ACTION_GETCONF = - "Get the configuration of an application"; + "Stop a running service"; String DESCRIBE_ACTION_KDIAG = "Diagnose Kerberos problems"; - String DESCRIBE_ACTION_KILL_CONTAINER = - "Kill a container in the application"; String DESCRIBE_ACTION_HELP = "Print help information"; String DESCRIBE_ACTION_LIST = - "List running Slider applications"; - String DESCRIBE_ACTION_LOOKUP = - "look up a YARN application"; - String DESCRIBE_ACTION_NODES = "List the node information for the YARN cluster or a running application"; - String DESCRIBE_ACTION_MONITOR = - "Monitor a running application"; + "List running services"; String DESCRIBE_ACTION_REGISTRY = - "Query the registry of a YARN application"; - String DESCRIBE_ACTION_RESOLVE = - "Resolve or list records in the YARN registry"; + "Query the registry of a service"; String DESCRIBE_ACTION_STATUS = - "Get the status of an application"; + "Get the status of a service"; String DESCRIBE_ACTION_THAW = - "Start a stopped application"; - String DESCRIBE_ACTION_VERSION = - "Print the Slider version information"; + "Start a service with pre-built specification or a previously stopped service"; String DESCRIBE_ACTION_CLIENT = "Install the application client in the specified directory or obtain a client keystore or truststore"; String DESCRIBE_ACTION_KEYTAB = "Manage a Kerberos keytab file (install, delete, list) in the sub-folder 'keytabs' of the user's Slider base directory"; - String DESCRIBE_ACTION_DIAGNOSTIC = "Diagnose the configuration of the running slider application and slider client"; String DESCRIBE_ACTION_RESOURCE = "Manage a file (install, delete, list) in the 'resources' sub-folder of the user's Slider base directory"; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/WaitArgsDelegate.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitArgsDelegate.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/WaitArgsDelegate.java index 1c27c01..86f3709 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitArgsDelegate.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/WaitArgsDelegate.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; import com.beust.jcommander.Parameter; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitTimeAccessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/WaitTimeAccessor.java similarity index 94% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitTimeAccessor.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/WaitTimeAccessor.java index 13d4d5a..f6afae6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitTimeAccessor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/client/params/WaitTimeAccessor.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.params; +package org.apache.hadoop.yarn.service.client.params; public interface WaitTimeAccessor { int getWaittime(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/compinstance/ComponentInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/compinstance/ComponentInstance.java index dcb455f..982a114 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/compinstance/ComponentInstance.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/compinstance/ComponentInstance.java @@ -36,17 +36,17 @@ import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.service.ServiceScheduler; +import org.apache.hadoop.yarn.service.api.records.ContainerState; import org.apache.hadoop.yarn.service.component.Component; import org.apache.hadoop.yarn.state.InvalidStateTransitionException; import org.apache.hadoop.yarn.state.SingleArcTransition; import org.apache.hadoop.yarn.state.StateMachine; import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.util.BoundedAppender; -import org.apache.slider.api.resource.ContainerState; -import org.apache.slider.common.tools.SliderUtils; +import org.apache.hadoop.yarn.service.utils.SliderUtils; import org.apache.hadoop.yarn.service.timelineservice.ServiceTimelinePublisher; -import org.apache.slider.server.servicemonitor.ProbeStatus; -import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders; +import org.apache.hadoop.yarn.service.servicemonitor.probe.ProbeStatus; +import org.apache.hadoop.yarn.service.registry.YarnRegistryViewForProviders; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,7 +88,7 @@ private volatile ContainerStatus status; private long containerStartedTime = 0; // This container object is used for rest API query - private org.apache.slider.api.resource.Container containerSpec; + private org.apache.hadoop.yarn.service.api.records.Container containerSpec; private static final StateMachineFactory @@ -142,11 +142,11 @@ public ComponentInstance(Component component, compInstance.getContainerId(), compInstance), 0, 1, TimeUnit.SECONDS); - org.apache.slider.api.resource.Container container = - new org.apache.slider.api.resource.Container(); + org.apache.hadoop.yarn.service.api.records.Container container = + new org.apache.hadoop.yarn.service.api.records.Container(); container.setId(compInstance.getContainerId().toString()); container.setLaunchTime(new Date()); - container.setState(org.apache.slider.api.resource.ContainerState.RUNNING_BUT_UNREADY); + container.setState(ContainerState.RUNNING_BUT_UNREADY); container.setBareHost(compInstance.container.getNodeId().getHost()); container.setComponentName(compInstance.getCompInstanceName()); if (compInstance.containerSpec != null) { @@ -290,7 +290,7 @@ public ContainerStatus getContainerStatus() { public void updateContainerStatus(ContainerStatus status) { this.status = status; - org.apache.slider.api.resource.Container container = + org.apache.hadoop.yarn.service.api.records.Container container = getCompSpec().getContainer(getContainerId().toString()); if (container != null) { container.setIp(StringUtils.join(",", status.getIPs())); @@ -330,7 +330,7 @@ public NodeId getNodeId() { return this.container.getNodeId(); } - public org.apache.slider.api.resource.Component getCompSpec() { + public org.apache.hadoop.yarn.service.api.records.Component getCompSpec() { return component.getComponentSpec(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java index bfe40c0..331871a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java @@ -39,9 +39,9 @@ import org.apache.hadoop.yarn.state.StateMachine; import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.util.Apps; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.server.servicemonitor.MonitorUtils; -import org.apache.slider.server.servicemonitor.Probe; +import org.apache.hadoop.yarn.service.utils.SliderUtils; +import org.apache.hadoop.yarn.service.servicemonitor.probe.MonitorUtils; +import org.apache.hadoop.yarn.service.servicemonitor.probe.Probe; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,12 +61,12 @@ import static org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEventType.STARTED; import static org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEventType.STOP; import static org.apache.hadoop.yarn.service.component.ComponentState.*; -import static org.apache.slider.api.ResourceKeys.CONTAINER_FAILURE_THRESHOLD; +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.CONTAINER_FAILURE_THRESHOLD; public class Component implements EventHandler { private static final Logger LOG = LoggerFactory.getLogger(Component.class); - private org.apache.slider.api.resource.Component componentSpec; + private org.apache.hadoop.yarn.service.api.records.Component componentSpec; private long allocateId; private Priority priority; private ServiceMetrics componentMetrics; @@ -124,7 +124,8 @@ FLEX, new FlexComponentTransition()) .installTopology(); - public Component(org.apache.slider.api.resource.Component component, + public Component( + org.apache.hadoop.yarn.service.api.records.Component component, long allocateId, ServiceContext context) { this.allocateId = allocateId; this.priority = Priority.newInstance((int) allocateId); @@ -418,7 +419,7 @@ public int getNumDesiredInstances() { return compInstances; } - public org.apache.slider.api.resource.Component getComponentSpec() { + public org.apache.hadoop.yarn.service.api.records.Component getComponentSpec() { return this.componentSpec; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/RestApiConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/RestApiConstants.java new file mode 100644 index 0000000..cbbb206 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/RestApiConstants.java @@ -0,0 +1,43 @@ +/* + * 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.conf; + +public interface RestApiConstants { + + // Rest endpoints + String CONTEXT_ROOT = "/services/v1"; + String VERSION = "/version"; + String APP_ROOT_PATH = "/applications"; + String APP_PATH = "/applications/{app_name}"; + String COMPONENT_PATH = "/applications/{app_name}/components/{component_name}"; + + // Query param + String APP_NAME = "app_name"; + String COMPONENT_NAME = "component_name"; + + String DEFAULT_COMPONENT_NAME = "default"; + + String PROPERTY_REST_SERVICE_HOST = "REST_SERVICE_HOST"; + String PROPERTY_REST_SERVICE_PORT = "REST_SERVICE_PORT"; + Long DEFAULT_UNLIMITED_LIFETIME = -1l; + + Integer ERROR_CODE_APP_DOES_NOT_EXIST = 404001; + Integer ERROR_CODE_APP_IS_NOT_RUNNING = 404002; + Integer ERROR_CODE_APP_SUBMITTED_BUT_NOT_RUNNING_YET = 404003; + Integer ERROR_CODE_APP_NAME_INVALID = 404004; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/SliderExitCodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/SliderExitCodes.java index d63c1a4..bdef600 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/SliderExitCodes.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/SliderExitCodes.java @@ -18,7 +18,7 @@ package org.apache.hadoop.yarn.service.conf; -import org.apache.slider.core.main.LauncherExitCodes; +import org.apache.hadoop.yarn.service.exceptions.LauncherExitCodes; public interface SliderExitCodes extends LauncherExitCodes { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/SliderKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/SliderKeys.java deleted file mode 100644 index e1687d2..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/SliderKeys.java +++ /dev/null @@ -1,195 +0,0 @@ -/* - * 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.conf; - -/** - * Keys and various constants for Slider - */ -public interface SliderKeys extends SliderXmlConfKeys { - - /** - * This is the name of the slider appmaster in configurations :{@value} - */ - String COMPONENT_AM = "slider-appmaster"; - - /** - * Slider role is "special":{@value} - */ - int ROLE_AM_PRIORITY_INDEX = 0; - - - /** - * The path under which cluster and temp data are stored - * {@value} - */ - String SLIDER_BASE_DIRECTORY = ".slider"; - - /** - * The paths under which Slider AM dependency libraries are stored - */ - String SLIDER_DEPENDENCY_LOCALIZED_DIR_LINK = "slider_dep"; - String SLIDER_DEPENDENCY_HDP_PARENT_DIR = "/hdp"; - String SLIDER_DEPENDENCY_DIR = "/apps/%s/slider"; - String SLIDER_DEPENDENCY_TAR_GZ_FILE_NAME = "slider-dep"; - String SLIDER_DEPENDENCY_TAR_GZ_FILE_EXT = ".tar.gz"; - String SLIDER_DEPENDENCY_DIR_PERMISSIONS = "755"; - - /** - * - */ - String HDP_VERSION_PROP_NAME = "HDP_VERSION"; - - /** - * name of the relative path to expaned an image into: {@value}. - * The title of this path is to help people understand it when - * they see it in their error messages - */ - String LOCAL_TARBALL_INSTALL_SUBDIR = "expandedarchive"; - - - /** - * Application type for YARN {@value} - */ - String APP_TYPE = "org-apache-slider"; - - /** - * A component type for an external app that has been predefined using the - * slider build command - */ - String COMPONENT_SEPARATOR = "-"; - - /** - * A component type for a client component - */ - String COMPONENT_TYPE_CLIENT = "client"; - - /** - * Key for application version. - */ - String APP_VERSION_UNKNOWN = "awaiting heartbeat..."; - - /** - * Keys for application container specific properties, like release timeout - */ - String APP_CONTAINER_RELEASE_TIMEOUT = "site.global.app_container.release_timeout_secs"; - - /** - * Subdirectories of HDFS cluster dir. - */ - String DATA_DIR_NAME = "data"; - String HISTORY_DIR_NAME = "history"; - String HISTORY_FILENAME_SUFFIX = "json"; - String HISTORY_FILENAME_PREFIX = "rolehistory-"; - String KEYTAB_DIR = "keytabs"; - String RESOURCE_DIR = "resources"; - - /** - * Filename pattern is required to save in strict temporal order. - * Important: older files must sort less-than newer files when using - * case-sensitive name sort. - */ - String HISTORY_FILENAME_CREATION_PATTERN = HISTORY_FILENAME_PREFIX +"%016x."+ - HISTORY_FILENAME_SUFFIX; - /** - * The posix regexp used to locate this - */ - String HISTORY_FILENAME_MATCH_PATTERN = HISTORY_FILENAME_PREFIX +"[0-9a-f]+\\."+ - HISTORY_FILENAME_SUFFIX; - /** - * The posix regexp used to locate this - */ - String HISTORY_FILENAME_GLOB_PATTERN = HISTORY_FILENAME_PREFIX +"*."+ - HISTORY_FILENAME_SUFFIX; - - String CLUSTER_DIRECTORY = "cluster"; - - /** - * JVM property to define the slider lib directory; - * this is set by the slider script: {@value} - */ - String PROPERTY_LIB_DIR = "slider.libdir"; - - /** - * name of generated dir for this conf: {@value} - */ - String SUBMITTED_CONF_DIR = "conf"; - - /** - * Slider AM log4j file name : {@value} - */ - String LOG4J_SERVER_PROP_FILENAME = "slideram-log4j.properties"; - - /** - * Log4j sysprop to name the resource :{@value} - */ - String SYSPROP_LOG4J_CONFIGURATION = "log4j.configuration"; - - /** - * sysprop for Slider AM log4j directory :{@value} - */ - String SYSPROP_LOG_DIR = "LOG_DIR"; - - /** - * name of the Slider client resource - * loaded when the service is loaded. - */ - String SLIDER_CLIENT_XML = "slider-client.xml"; - - /** - * The name of the resource to put on the classpath - */ - String SLIDER_SERVER_XML = "slider-server.xml"; - - String TMP_DIR_PREFIX = "tmp"; - - /** - * Store the default app definition, e.g. metainfo file or content of a folder - */ - String APP_DEF_DIR = "appdef"; - /** - * Store additional app defs - co-processors - */ - String ADDONS_DIR = "addons"; - - String SLIDER_JAR = "slider-core.jar"; - - String STDOUT_AM = "slider-out.txt"; - String STDERR_AM = "slider-err.txt"; - - String HADOOP_USER_NAME = "HADOOP_USER_NAME"; - - /** - * Name of the AM filter to use: {@value} - */ - String AM_FILTER_NAME = - "org.apache.hadoop.yarn.server.webproxy.amfilter.AmFilterInitializer"; - - String YARN_CONTAINER_PATH = "/node/container/"; - - String APP_CONF_DIR = "conf"; - - String APP_LIB_DIR = "lib"; - - String OUT_FILE = "stdout.txt"; - String ERR_FILE = "stderr.txt"; - - String QUICK_LINKS = "quicklinks"; - - String KEY_CONTAINER_LAUNCH_DELAY = "container.launch.delay.sec"; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/SliderXmlConfKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/SliderXmlConfKeys.java deleted file mode 100644 index 523e08d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/SliderXmlConfKeys.java +++ /dev/null @@ -1,191 +0,0 @@ -/* - * 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.conf; - -/** - * These are the keys that can be added to conf/slider-client.xml. - */ -public interface SliderXmlConfKeys { - String PREFIX_PROVIDER = "slider.provider"; - /** - * pattern to identify a provider - * {@value} - */ - String KEY_PROVIDER = PREFIX_PROVIDER + ".%s"; - - /** - * conf option set to point to where the config came from - * {@value} - */ - String KEY_TEMPLATE_ORIGIN = "slider.template.origin"; - - /** - * Original name for the default FS. This is still - * expected by applications deployed - */ - String FS_DEFAULT_NAME_CLASSIC = "fs.default.name"; - - /** - * Slider principal - */ - String KEY_KERBEROS_PRINCIPAL = "slider.kerberos.principal"; - - /** - * Name of the property for ACLs for Slider AM. - * {@value} - */ - String KEY_PROTOCOL_ACL = "slider.security.protocol.acl"; - - /** - * Limit on restarts for the AM - * {@value} - */ - String KEY_AM_RESTART_LIMIT = "slider.yarn.restart.limit"; - - /** - * queue name, by default let YARN pick the queue - */ - String KEY_YARN_QUEUE = "slider.yarn.queue"; - String DEFAULT_YARN_QUEUE = null; - - /** - * default priority - */ - String KEY_YARN_QUEUE_PRIORITY = "slider.yarn.queue.priority"; - int DEFAULT_YARN_QUEUE_PRIORITY = 1; - - - String KEY_AM_RESOURCE_MEM = "slider.am.resource.memory"; - long DEFAULT_KEY_AM_RESOURCE_MEM = 1024; - - /** - * The slider base path: {@value} - * Defaults to HomeDir/.slider - */ - String KEY_SLIDER_BASE_PATH = "slider.base.path"; - - - /** - * Option for the permissions for the cluster directory itself: {@value} - */ - String CLUSTER_DIRECTORY_PERMISSIONS = - "slider.cluster.directory.permissions"; - - /** - * Default value for the permissions :{@value} - */ - String DEFAULT_CLUSTER_DIRECTORY_PERMISSIONS = "750"; - - /** - * - * Option for the permissions for the data directory itself: {@value} - */ - String DATA_DIRECTORY_PERMISSIONS = "slider.data.directory.permissions"; - - /** - * Default value for the data directory permissions: {@value} - */ - String DEFAULT_DATA_DIRECTORY_PERMISSIONS = "750"; - - String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH = - "ipc.client.fallback-to-simple-auth-allowed"; - String HADOOP_HTTP_FILTER_INITIALIZERS = - "hadoop.http.filter.initializers"; - String KEY_KEYSTORE_LOCATION = "ssl.server.keystore.location"; - String KEY_AM_LOGIN_KEYTAB_NAME = "slider.am.login.keytab.name"; - /** Declare that a keytab must be provided */ - String KEY_AM_LOGIN_KEYTAB_REQUIRED = "slider.am.login.keytab.required"; - String KEY_HDFS_KEYTAB_DIR = "slider.hdfs.keytab.dir"; - String KEY_AM_KEYTAB_LOCAL_PATH = "slider.am.keytab.local.path"; - String KEY_KEYTAB_PRINCIPAL = "slider.keytab.principal.name"; - String KEY_SECURITY_ENABLED = "site.global.security_enabled"; - - /** - * Set to disable server-side checks for python, openssl &c. - * This should only be set for testing - */ - String KEY_SLIDER_AM_DEPENDENCY_CHECKS_DISABLED = - "slider.am.dependency.checks.disabled"; - - /** - * The path to the python executable utilized to launch the agent. - */ - String PYTHON_EXECUTABLE_PATH = "agent.python.exec.path"; - - /** - * Flag to enable the insecure AM filter: {@value} - */ - String X_DEV_INSECURE_WS = "slider.feature.ws.insecure"; - - /** - * Flag to indicate the insecure AM filter is enabled by default: {@value}. - */ - boolean X_DEV_INSECURE_DEFAULT = false; - - - /** - * Flag to indicate the insecure AM filter is required for - * complex REST Verbs: {@value}. - * When Slider switches to being Hadoop 2.7+ only, this flag - * can be set to false - */ - boolean X_DEV_INSECURE_REQUIRED = true; - - /** - * - */ - String KEY_IPC_CLIENT_RETRY_POLICY_ENABLED = - "slider.ipc.client.retry.enabled"; - boolean IPC_CLIENT_RETRY_POLICY_ENABLED_DEFAULT = true; - String KEY_IPC_CLIENT_RETRY_POLICY_SPEC = - "slider.ipc.client.retry.policy.spec"; - String IPC_CLIENT_RETRY_POLICY_SPEC_DEFAULT = - "10000,6,60000,10"; //t1,n1,t2,n2,... - - String KEY_AM_LAUNCH_ENV = "slider.am.launch.env"; - - /** - * From {@code DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY} - */ - String DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY = "dfs.namenode.kerberos.principal"; - - String DFS_DATANODE_KERBEROS_PRINCIPAL_KEY = "dfs.datanode.kerberos.principal"; - - //Delegation token related keys - String DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_KEY - = "dfs.namenode.delegation.key.update-interval"; - long DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT = 24 * 60 * 60 * - 1000; // 1 day - String DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY - = "dfs.namenode.delegation.token.renew-interval"; - long DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT = 24 * 60 * 60 * - 1000; // 1 day - String DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY - = "dfs.namenode.delegation.token.max-lifetime"; - long DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT = 7 * 24 * 60 * 60 * - 1000; // 7 days - String DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY - = "dfs.namenode.delegation.token.always-use"; // for tests - boolean DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT = false; - String DFS_NAMENODE_KEYTAB_FILE_KEY = "dfs.namenode.keytab.file"; - String DFS_NAMENODE_DU_RESERVED_KEY = "dfs.namenode.resource.du.reserved"; - - String MAPREDUCE_JOB_CREDENTIALS_BINARY = "mapreduce.job.credentials.binary"; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConf.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConf.java index 9225570..33fc671 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConf.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConf.java @@ -18,7 +18,7 @@ package org.apache.hadoop.yarn.service.conf; -import org.apache.slider.api.resource.Configuration; +import org.apache.hadoop.yarn.service.api.records.Configuration; public class YarnServiceConf { @@ -30,8 +30,54 @@ public static final String CONTAINER_RETRY_MAX = "yarn.service.container-failure.retry.max"; public static final String CONTAINER_RETRY_INTERVAL = "yarn.service.container-failure.retry-interval"; + public static final String AM_RESTART_MAX = "yarn.service.am-restart.max-attempts"; + public static final String AM_RESOURCE_MEM = "yarn.service.am-resource.memory"; + public static final long DEFAULT_KEY_AM_RESOURCE_MEM = 1024; + + public static final String YARN_QUEUE = "yarn.service.queue"; + + /** + * The yarn service base path: + * Defaults to HomeDir/.yarn/ + */ + public static final String YARN_SERVICE_BASE_PATH = "yarn.service.base.path"; + + //TODO rename + /** Declare that a keytab must be provided */ + public static final String KEY_AM_LOGIN_KEYTAB_REQUIRED = "slider.am.login.keytab.required"; + public static final String KEY_AM_LOGIN_KEYTAB_NAME = "slider.am.login.keytab.name"; + public static final String KEY_HDFS_KEYTAB_DIR = "slider.hdfs.keytab.dir"; + public static final String KEY_AM_KEYTAB_LOCAL_PATH = "slider.am.keytab.local.path"; + + /** + * maximum number of failed containers (in a single component) + * before the app exits + */ + public static final String CONTAINER_FAILURE_THRESHOLD = + "yarn.service.container-failure-per-component.threshold"; + /** + * Maximum number of container failures on a node before the node is blacklisted + */ + public static final String NODE_BLACKLIST_THRESHOLD = + "yarn.service.node-blacklist.threshold"; + + /** + * The failure count for CONTAINER_FAILURE_THRESHOLD and NODE_BLACKLIST_THRESHOLD + * gets reset periodically, the unit is seconds. + */ + public static final String CONTAINER_FAILURE_WINDOW = + "yarn.service.failure-count-reset.window"; + + /** + * interval between readiness checks. + */ + public static final String READINESS_CHECK_INTERVAL = "yarn.service.readiness-check-interval.seconds"; + public static final int DEFAULT_READINESS_CHECK_INTERVAL = 30; // seconds + /** - * Get long value for the property + * Get long value for the property. First get from the userConf, if not + * present, get from systemConf. + * * @param name name of the property * @param defaultValue default value of the property, if it is not defined in * userConf and systemConf. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConstants.java new file mode 100644 index 0000000..cbcba82 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConstants.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.service.conf; + +public interface YarnServiceConstants { + + /** + * The path under which cluster and temp data are stored + */ + String SERVICE_BASE_DIRECTORY = ".yarn"; + + /** + * The paths under which Service AM dependency libraries are stored + */ + String DEPENDENCY_LOCALIZED_DIR_LINK = "service_dep"; + String DEPENDENCY_DIR = "/yarn-services/%s/"; + String DEPENDENCY_TAR_GZ_FILE_NAME = "service-dep"; + String DEPENDENCY_TAR_GZ_FILE_EXT = ".tar.gz"; + String DEPENDENCY_DIR_PERMISSIONS = "755"; + + /** + * Application type for YARN service + */ + String APP_TYPE = "yarn-service"; + + String KEYTAB_DIR = "keytabs"; + String RESOURCE_DIR = "resources"; + + + String SERVICES_DIRECTORY = "services"; + + /** + * JVM property to define the service lib directory; + * this is set by the yarn.sh script + */ + String PROPERTY_LIB_DIR = "service.libdir"; + + /** + * name of generated dir for this conf + */ + String SUBMITTED_CONF_DIR = "conf"; + + /** + * Service AM log4j file name + */ + String YARN_SERVICE_LOG4J_FILENAME = "yarnservice-log4j.properties"; + + /** + * Log4j sysprop to name the resource + */ + String SYSPROP_LOG4J_CONFIGURATION = "log4j.configuration"; + + /** + * sysprop for Service AM log4j directory + */ + String SYSPROP_LOG_DIR = "LOG_DIR"; + + String TMP_DIR_PREFIX = "tmp"; + + + String SERVICE_CORE_JAR = "yarn-service-core.jar"; + + String STDOUT_AM = "serviceam-out.txt"; + String STDERR_AM = "serviceam-err.txt"; + + String HADOOP_USER_NAME = "HADOOP_USER_NAME"; + + String APP_CONF_DIR = "conf"; + + String APP_LIB_DIR = "lib"; + + String OUT_FILE = "stdout.txt"; + String ERR_FILE = "stderr.txt"; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/AbstractLauncher.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/AbstractLauncher.java index a3e1bf2..e4eae20 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/AbstractLauncher.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.launch; +package org.apache.hadoop.yarn.service.containerlaunch; import com.google.common.base.Preconditions; import org.apache.hadoop.security.Credentials; @@ -26,9 +26,9 @@ import org.apache.hadoop.yarn.api.records.ContainerRetryPolicy; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.util.Records; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.slider.common.tools.CoreFileSystem; -import org.apache.slider.common.tools.SliderUtils; +import org.apache.hadoop.yarn.service.conf.YarnServiceConstants; +import org.apache.hadoop.yarn.service.utils.CoreFileSystem; +import org.apache.hadoop.yarn.service.utils.SliderUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -220,7 +220,7 @@ private void dumpLocalResources() { protected void propagateUsernameInInsecureCluster() throws IOException { //insecure cluster: propagate user name via env variable String userName = UserGroupInformation.getCurrentUser().getUserName(); - envVars.put(SliderKeys.HADOOP_USER_NAME, userName); + envVars.put(YarnServiceConstants.HADOOP_USER_NAME, userName); } /** diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ClasspathConstructor.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ClasspathConstructor.java index 6eb4058..22b3877 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ClasspathConstructor.java @@ -16,13 +16,13 @@ * limitations under the License. */ -package org.apache.slider.core.launch; +package org.apache.hadoop.yarn.service.containerlaunch; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.slider.common.tools.SliderUtils; +import org.apache.hadoop.yarn.service.utils.SliderUtils; import java.util.ArrayList; import java.util.Arrays; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CommandLineBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CommandLineBuilder.java similarity index 93% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CommandLineBuilder.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CommandLineBuilder.java index 5ab0532..7baa284 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CommandLineBuilder.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CommandLineBuilder.java @@ -16,11 +16,11 @@ * limitations under the License. */ -package org.apache.slider.core.launch; +package org.apache.hadoop.yarn.service.containerlaunch; import com.google.common.base.Preconditions; import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.slider.common.tools.SliderUtils; +import org.apache.hadoop.yarn.service.utils.SliderUtils; import java.util.ArrayList; import java.util.List; @@ -42,10 +42,7 @@ public void add(Object... args) { } } - /** - * Get the number of arguments - * @return an integer >= 0 - */ + // Get the number of arguments public int size() { return argumentList.size(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ContainerLaunchService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java similarity index 91% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ContainerLaunchService.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java index 2037a3b..fcbb69b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ContainerLaunchService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java @@ -16,17 +16,17 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.service; +package org.apache.hadoop.yarn.service.containerlaunch; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.service.api.records.Component; import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; import org.apache.hadoop.yarn.service.provider.ProviderService; import org.apache.hadoop.yarn.service.provider.ProviderFactory; -import org.apache.slider.api.resource.Application; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.core.launch.AbstractLauncher; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -81,7 +81,7 @@ public ContainerLauncher( } @Override public void run() { - org.apache.slider.api.resource.Component compSpec = instance.getCompSpec(); + Component compSpec = instance.getCompSpec(); ProviderService provider = ProviderFactory.getProviderService( compSpec.getArtifact()); AbstractLauncher launcher = new AbstractLauncher(fs, null); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CredentialUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CredentialUtils.java similarity index 83% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CredentialUtils.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CredentialUtils.java index 1fd49ab..fce58e5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CredentialUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CredentialUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.launch; +package org.apache.hadoop.yarn.service.containerlaunch; import com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; @@ -37,13 +37,11 @@ import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; import org.apache.hadoop.yarn.util.ConverterUtils; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.DataOutputStream; import java.io.File; -import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.IOException; import java.io.Serializable; @@ -55,9 +53,7 @@ import java.util.Date; import java.util.Iterator; import java.util.List; -import java.util.Map; -import static org.apache.hadoop.security.UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION; import static org.apache.hadoop.yarn.conf.YarnConfiguration.*; /** @@ -67,9 +63,6 @@ */ public final class CredentialUtils { - public static final String JOB_CREDENTIALS_BINARY - = SliderXmlConfKeys.MAPREDUCE_JOB_CREDENTIALS_BINARY; - private CredentialUtils() { } @@ -97,59 +90,6 @@ public static ByteBuffer marshallCredentials(Credentials credentials) throws IOE return buffer; } - public static File locateEnvCredentials(Map env, - Configuration conf, - StringBuffer sourceTextOut) throws FileNotFoundException { - String tokenFilename = env.get(HADOOP_TOKEN_FILE_LOCATION); - String source = "environment variable " + HADOOP_TOKEN_FILE_LOCATION; - if (tokenFilename == null) { - tokenFilename = conf.get(JOB_CREDENTIALS_BINARY); - source = "configuration option " + JOB_CREDENTIALS_BINARY; - } - if (tokenFilename != null) { - // use delegation tokens, i.e. from Oozie - File file = new File(tokenFilename.trim()); - String details = String.format( - "Token File %s from %s", - file, - source); - if (!file.exists()) { - throw new FileNotFoundException("No " + details); - } - if (!file.isFile() && !file.canRead()) { - throw new FileNotFoundException("Cannot read " + details); - } - sourceTextOut.append(details); - return file; - } else { - return null; - } - } - - /** - * Load the credentials from the environment. This looks at - * the value of {@link UserGroupInformation#HADOOP_TOKEN_FILE_LOCATION} - * and attempts to read in the value - * @param env environment to resolve the variable from - * @param conf configuration use when reading the tokens - * @return a set of credentials, or null if the environment did not - * specify any - * @throws IOException if a location for credentials was defined, but - * the credentials could not be loaded. - */ - public static Credentials loadTokensFromEnvironment(Map env, - Configuration conf) - throws IOException { - StringBuffer origin = new StringBuffer(); - File file = locateEnvCredentials(env, conf, origin); - if (file != null) { - LOG.debug("Using {}", origin); - return Credentials.readTokenStorageFile(file, conf); - } else { - return null; - } - } - /** * Save credentials to a file * @param file file to save to (will be overwritten) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/JavaCommandLineBuilder.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/JavaCommandLineBuilder.java index b8aa4c6..cbcb0d6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/JavaCommandLineBuilder.java @@ -16,16 +16,15 @@ * limitations under the License. */ -package org.apache.slider.core.launch; +package org.apache.hadoop.yarn.service.containerlaunch; import com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadConfigException; +import org.apache.hadoop.yarn.service.utils.SliderUtils; +import org.apache.hadoop.yarn.service.exceptions.BadConfigException; -import java.util.Iterator; import java.util.Map; /** diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadClusterStateException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadClusterStateException.java similarity index 91% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadClusterStateException.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadClusterStateException.java index e73ce57..db9de7a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadClusterStateException.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadClusterStateException.java @@ -16,8 +16,9 @@ * limitations under the License. */ -package org.apache.slider.core.exceptions; +package org.apache.hadoop.yarn.service.exceptions; +import org.apache.hadoop.yarn.service.exceptions.SliderException; /** * The system is in a bad state diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadCommandArgumentsException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadCommandArgumentsException.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadCommandArgumentsException.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadCommandArgumentsException.java index 0d5d686..41e3251 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadCommandArgumentsException.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadCommandArgumentsException.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.exceptions; +package org.apache.hadoop.yarn.service.exceptions; public class BadCommandArgumentsException extends SliderException { public BadCommandArgumentsException(String s, Object... args) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadConfigException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadConfigException.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadConfigException.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadConfigException.java index 65a8ea8..8199c3c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadConfigException.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadConfigException.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.exceptions; +package org.apache.hadoop.yarn.service.exceptions; /** * An exception to raise on a bad configuration diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ErrorStrings.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ErrorStrings.java index 8b04969..3577b59 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ErrorStrings.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.exceptions; +package org.apache.hadoop.yarn.service.exceptions; public interface ErrorStrings { String E_UNSTABLE_CLUSTER = "Unstable Application Instance :"; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ExitCodeProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ExitCodeProvider.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ExitCodeProvider.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ExitCodeProvider.java index 503b9b9..d66b860 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ExitCodeProvider.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ExitCodeProvider.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.main; +package org.apache.hadoop.yarn.service.exceptions; /** * Get the exit code of an exception. Making it an interface allows diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/LauncherExitCodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/LauncherExitCodes.java similarity index 99% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/LauncherExitCodes.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/LauncherExitCodes.java index 83e89f0..9657536 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/LauncherExitCodes.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/LauncherExitCodes.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.main; +package org.apache.hadoop.yarn.service.exceptions; /* * Common Exit codes diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/util/RestApiErrorMessages.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java similarity index 98% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/util/RestApiErrorMessages.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java index 74f7e06..7be23f3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/util/RestApiErrorMessages.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.slider.util; +package org.apache.hadoop.yarn.service.exceptions; public interface RestApiErrorMessages { String ERROR_APPLICATION_NAME_INVALID = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLaunchException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ServiceLaunchException.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLaunchException.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ServiceLaunchException.java index 27813b7..e83ccbe 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLaunchException.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ServiceLaunchException.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.main; +package org.apache.hadoop.yarn.service.exceptions; import org.apache.hadoop.yarn.exceptions.YarnException; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/SliderException.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderException.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/SliderException.java index 1430c5a..5b74b80 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderException.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/SliderException.java @@ -16,10 +16,9 @@ * limitations under the License. */ -package org.apache.slider.core.exceptions; +package org.apache.hadoop.yarn.service.exceptions; import org.apache.hadoop.yarn.service.conf.SliderExitCodes; -import org.apache.slider.core.main.ServiceLaunchException; public class SliderException extends ServiceLaunchException implements SliderExitCodes { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UsageException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/UsageException.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UsageException.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/UsageException.java index 8684294..3a9fa25 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UsageException.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/UsageException.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.exceptions; +package org.apache.hadoop.yarn.service.exceptions; /** * Used to raise a usage exception ... this has the exit code diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java index 6c91a13..0d11be2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java @@ -19,18 +19,12 @@ package org.apache.hadoop.yarn.service.provider; import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.registry.client.api.RegistryOperations; -import org.apache.slider.api.resource.Artifact; -import org.apache.slider.api.resource.ConfigFile; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.SliderException; -import org.codehaus.jettison.json.JSONObject; - -import java.io.File; +import org.apache.hadoop.yarn.service.api.records.Artifact; +import org.apache.hadoop.yarn.service.api.records.ConfigFile; +import org.apache.hadoop.yarn.service.utils.SliderUtils; + import java.io.IOException; import java.nio.file.Paths; import java.util.HashSet; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractProviderService.java index 6f9f5175..504680d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractProviderService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractProviderService.java @@ -20,14 +20,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.service.conf.YarnServiceConf; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.launch.AbstractLauncher; -import org.apache.slider.core.launch.CommandLineBuilder; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.conf.YarnServiceConstants; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; +import org.apache.hadoop.yarn.service.utils.SliderUtils; +import org.apache.hadoop.yarn.service.exceptions.SliderException; +import org.apache.hadoop.yarn.service.containerlaunch.AbstractLauncher; +import org.apache.hadoop.yarn.service.containerlaunch.CommandLineBuilder; import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; import org.apache.hadoop.yarn.service.ServiceContext; import org.slf4j.Logger; @@ -42,7 +42,7 @@ import static org.apache.hadoop.yarn.service.utils.ServiceApiUtil.$; public abstract class AbstractProviderService implements ProviderService, - SliderKeys { + YarnServiceConstants { protected static final Logger log = LoggerFactory.getLogger(AbstractProviderService.class); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderFactory.java index b53652a..83c9961 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderFactory.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderFactory.java @@ -19,8 +19,7 @@ package org.apache.hadoop.yarn.service.provider; import org.apache.hadoop.yarn.service.provider.defaultImpl.DefaultProviderFactory; -import org.apache.slider.api.resource.Artifact; -import org.apache.slider.core.exceptions.SliderException; +import org.apache.hadoop.yarn.service.api.records.Artifact; import org.apache.hadoop.yarn.service.provider.docker.DockerProviderFactory; import org.apache.hadoop.yarn.service.provider.tarball.TarballProviderFactory; import org.slf4j.Logger; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderService.java index 306620d..9ef0176 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderService.java @@ -19,10 +19,10 @@ package org.apache.hadoop.yarn.service.provider; import org.apache.hadoop.conf.Configuration; -import org.apache.slider.api.resource.Application; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.launch.AbstractLauncher; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; +import org.apache.hadoop.yarn.service.exceptions.SliderException; +import org.apache.hadoop.yarn.service.containerlaunch.AbstractLauncher; import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; import java.io.IOException; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java index 647bfe9..a044838 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java @@ -24,30 +24,24 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; -import org.apache.slider.api.ClusterNode; -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.api.RoleKeys; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.api.resource.ConfigFile; -import org.apache.slider.api.resource.Configuration; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.launch.AbstractLauncher; -import org.apache.slider.core.launch.ContainerLauncher; -import org.apache.slider.core.registry.docstore.ConfigFormat; -import org.apache.slider.core.registry.docstore.PublishedConfiguration; -import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter; -import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; import org.apache.hadoop.yarn.service.ServiceContext; -import org.apache.slider.server.appmaster.state.StateAccessForProviders; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.api.records.ConfigFile; +import org.apache.hadoop.yarn.service.api.records.ConfigFormat; +import org.apache.hadoop.yarn.service.api.records.Configuration; +import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; +import org.apache.hadoop.yarn.service.conf.YarnServiceConstants; +import org.apache.hadoop.yarn.service.conf.YarnServiceConf; +import org.apache.hadoop.yarn.service.containerlaunch.AbstractLauncher; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.SliderException; +import org.apache.hadoop.yarn.service.utils.PublishedConfiguration; +import org.apache.hadoop.yarn.service.utils.PublishedConfigurationOutputter; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; +import org.apache.hadoop.yarn.service.utils.SliderUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,23 +49,18 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Collection; import java.util.HashMap; -import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.regex.Pattern; -import static org.apache.slider.api.ServiceApiConstants.*; -import static org.apache.hadoop.yarn.service.utils.ServiceApiUtil.$; +import static org.apache.hadoop.yarn.service.api.constants.ServiceApiConstants.*; /** * This is a factoring out of methods handy for providers. It's bonded to a log * at construction time. */ -public class ProviderUtils implements RoleKeys, SliderKeys { +public class ProviderUtils implements YarnServiceConstants { protected static final Logger log = LoggerFactory.getLogger(ProviderUtils.class); @@ -174,46 +163,22 @@ public static void substituteMapWithTokens(Map configs, } /** - * Get resource requirements from a String value. If value isn't specified, - * use the default value. If value is greater than max, use the max value. - * @param val string value - * @param defVal default value - * @param maxVal maximum value - * @return int resource requirement - */ - public int getRoleResourceRequirement(String val, - int defVal, - int maxVal) { - if (val==null) { - val = Integer.toString(defVal); - } - Integer intVal; - if (ResourceKeys.YARN_RESOURCE_MAX.equals(val)) { - intVal = maxVal; - } else { - intVal = Integer.decode(val); - } - return intVal; - } - - - /** * Localize the service keytabs for the application. * @param launcher container launcher * @param fileSystem file system * @throws IOException trouble uploading to HDFS */ - public void localizeServiceKeytabs(ContainerLauncher launcher, + public void localizeServiceKeytabs(AbstractLauncher launcher, SliderFileSystem fileSystem, Application application) throws IOException { Configuration conf = application.getConfiguration(); String keytabPathOnHost = - conf.getProperty(SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH); + conf.getProperty(YarnServiceConf.KEY_AM_KEYTAB_LOCAL_PATH); if (SliderUtils.isUnset(keytabPathOnHost)) { String amKeytabName = - conf.getProperty(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME); + conf.getProperty(YarnServiceConf.KEY_AM_LOGIN_KEYTAB_NAME); String keytabDir = - conf.getProperty(SliderXmlConfKeys.KEY_HDFS_KEYTAB_DIR); + conf.getProperty(YarnServiceConf.KEY_HDFS_KEYTAB_DIR); // we need to localize the keytab files in the directory Path keytabDirPath = fileSystem.buildKeytabPath(keytabDir, null, application.getName()); @@ -434,38 +399,4 @@ private static void resolvePlainTemplateAndSaveOnHdfs(FileSystem fs, String.valueOf(instance.getCompInstanceId().getId())); return tokens; } - - /** - * Add ROLE_HOST tokens for substitution into config values. - * @param tokens existing tokens - * @param amState access to AM state - */ - public static void addComponentHostTokens(Map tokens, - StateAccessForProviders amState) { - if (amState == null) { - return; - } - for (Map.Entry> entry : - amState.getRoleClusterNodeMapping().entrySet()) { - String tokenName = entry.getKey().toUpperCase(Locale.ENGLISH) + "_HOST"; - String hosts = StringUtils .join(",", - getHostsList(entry.getValue().values(), true)); - tokens.put($(tokenName), hosts); - } - } - - /** - * Return a list of hosts based on current ClusterNodes. - * @param values cluster nodes - * @param hostOnly whether host or host/server name will be added to list - * @return list of hosts - */ - public static Iterable getHostsList(Collection values, - boolean hostOnly) { - List hosts = new ArrayList<>(); - for (ClusterNode cn : values) { - hosts.add(hostOnly ? cn.host : cn.host + "/" + cn.name); - } - return hosts; - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java index 32cedb6..0920a9c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java @@ -19,8 +19,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.yarn.service.provider.AbstractClientProvider; -import org.apache.slider.api.resource.Artifact; -import org.apache.slider.api.resource.ConfigFile; +import org.apache.hadoop.yarn.service.api.records.Artifact; +import org.apache.hadoop.yarn.service.api.records.ConfigFile; import java.io.IOException; import java.nio.file.Paths; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultProviderService.java index a77214c..33f8278 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultProviderService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultProviderService.java @@ -19,9 +19,9 @@ import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; import org.apache.hadoop.yarn.service.provider.AbstractProviderService; -import org.apache.slider.api.resource.Application; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.core.launch.AbstractLauncher; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; +import org.apache.hadoop.yarn.service.containerlaunch.AbstractLauncher; import java.io.IOException; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java index c1f225c..d4a2254 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java @@ -19,16 +19,16 @@ import org.apache.commons.lang.StringUtils; import org.apache.hadoop.fs.FileSystem; -import org.apache.slider.api.resource.Artifact; -import org.apache.slider.api.resource.ConfigFile; -import org.apache.hadoop.yarn.service.conf.SliderKeys; +import org.apache.hadoop.yarn.service.api.records.Artifact; +import org.apache.hadoop.yarn.service.api.records.ConfigFile; +import org.apache.hadoop.yarn.service.conf.YarnServiceConstants; import org.apache.hadoop.yarn.service.provider.AbstractClientProvider; -import org.apache.slider.util.RestApiErrorMessages; +import org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages; import java.io.IOException; public class DockerClientProvider extends AbstractClientProvider - implements SliderKeys { + implements YarnServiceConstants { public DockerClientProvider() { super(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerProviderService.java index c20eaad..236ddd9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerProviderService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerProviderService.java @@ -21,9 +21,9 @@ import org.apache.hadoop.registry.client.binding.RegistryUtils; import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; import org.apache.hadoop.yarn.service.provider.AbstractProviderService; -import org.apache.slider.api.resource.Application; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.core.launch.AbstractLauncher; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; +import org.apache.hadoop.yarn.service.containerlaunch.AbstractLauncher; import java.io.IOException; import java.text.MessageFormat; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java index 2b54be9..01f7b20 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java @@ -20,17 +20,17 @@ import org.apache.commons.lang.StringUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.slider.api.resource.Artifact; -import org.apache.slider.api.resource.ConfigFile; -import org.apache.hadoop.yarn.service.conf.SliderKeys; +import org.apache.hadoop.yarn.service.api.records.Artifact; +import org.apache.hadoop.yarn.service.api.records.ConfigFile; +import org.apache.hadoop.yarn.service.conf.YarnServiceConstants; import org.apache.hadoop.yarn.service.provider.AbstractClientProvider; -import org.apache.slider.util.RestApiErrorMessages; +import org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages; import java.io.IOException; import java.nio.file.Paths; public class TarballClientProvider extends AbstractClientProvider - implements SliderKeys { + implements YarnServiceConstants { public TarballClientProvider() { } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballProviderService.java index 3c3d425..2403255 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballProviderService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballProviderService.java @@ -22,9 +22,9 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; import org.apache.hadoop.yarn.service.provider.AbstractProviderService; -import org.apache.slider.api.resource.Application; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.core.launch.AbstractLauncher; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; +import org.apache.hadoop.yarn.service.containerlaunch.AbstractLauncher; import java.io.IOException; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/registry/CustomRegistryConstants.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/registry/CustomRegistryConstants.java index 13ad5c5..56634f6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/registry/CustomRegistryConstants.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.registry.info; +package org.apache.hadoop.yarn.service.registry; /** * These are constants unique to the Slider AM diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/registry/YarnRegistryViewForProviders.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/registry/YarnRegistryViewForProviders.java index 76ce7a5..ef5ed91 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/registry/YarnRegistryViewForProviders.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.server.services.yarnregistry; +package org.apache.hadoop.yarn.service.registry; import com.google.common.base.Preconditions; import org.apache.commons.logging.Log; @@ -30,9 +30,8 @@ import org.apache.hadoop.registry.client.binding.RegistryPathUtils; import org.apache.hadoop.registry.client.types.ServiceRecord; -import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; import org.apache.hadoop.yarn.service.compinstance.ComponentInstanceId; -import org.apache.slider.common.tools.SliderUtils; +import org.apache.hadoop.yarn.service.utils.SliderUtils; import java.io.IOException; import java.util.List; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/BaseRestClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/BaseRestClient.java similarity index 94% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/BaseRestClient.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/BaseRestClient.java index d936a22..2d01bef 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/BaseRestClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/BaseRestClient.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.client.rest; +package org.apache.hadoop.yarn.service.rest; import com.google.common.base.Preconditions; import com.sun.jersey.api.client.Client; @@ -24,9 +24,6 @@ import com.sun.jersey.api.client.GenericType; import com.sun.jersey.api.client.UniformInterfaceException; import com.sun.jersey.api.client.WebResource; -import org.apache.slider.core.exceptions.ExceptionConverter; -import org.apache.slider.core.restclient.HttpVerb; -import org.apache.slider.core.restclient.UgiJerseyBinding; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -141,7 +138,7 @@ public WebResource resource(URI u) { /** * Create a Web resource from the client. * - * @param u the URI of the resource. + * @param url the URI of the resource. * @return the Web resource. */ diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/ExceptionConverter.java similarity index 99% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/ExceptionConverter.java index efec676..12fdc79 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/ExceptionConverter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.exceptions; +package org.apache.hadoop.yarn.service.rest; import com.sun.jersey.api.client.ClientHandlerException; import com.sun.jersey.api.client.ClientResponse; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/HttpVerb.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/HttpVerb.java index c040345..93f9082 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/HttpVerb.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.restclient; +package org.apache.hadoop.yarn.service.rest; /** * Http verbs with details on what they support in terms of submit and diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/SliderURLConnectionFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/SliderURLConnectionFactory.java similarity index 99% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/SliderURLConnectionFactory.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/SliderURLConnectionFactory.java index e453f52..fcd7f55 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/SliderURLConnectionFactory.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/SliderURLConnectionFactory.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.restclient; +package org.apache.hadoop.yarn.service.rest; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.web.KerberosUgiAuthenticator; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UgiJerseyBinding.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/UgiJerseyBinding.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UgiJerseyBinding.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/UgiJerseyBinding.java index bf71861..b3fdef9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UgiJerseyBinding.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/UgiJerseyBinding.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.restclient; +package org.apache.hadoop.yarn.service.rest; import com.google.common.base.Preconditions; import com.sun.jersey.api.client.Client; @@ -28,7 +28,6 @@ import com.sun.jersey.client.urlconnection.URLConnectionClientHandler; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.authentication.client.AuthenticationException; -import org.apache.slider.core.exceptions.ExceptionConverter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/UrlConnectionOperations.java similarity index 88% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/UrlConnectionOperations.java index 46f0d02..d7f768e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/rest/UrlConnectionOperations.java @@ -16,24 +16,17 @@ * limitations under the License. */ -package org.apache.slider.core.restclient; +package org.apache.hadoop.yarn.service.rest; import com.google.common.base.Preconditions; -import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authentication.client.AuthenticationException; -import org.apache.hadoop.yarn.webapp.ForbiddenException; -import org.apache.hadoop.yarn.webapp.NotFoundException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.net.ssl.SSLException; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.net.HttpURLConnection; import java.net.URL; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/ServiceMonitor.java similarity index 85% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMonitor.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/ServiceMonitor.java index bc37614..98a76ea 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMonitor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/ServiceMonitor.java @@ -16,21 +16,20 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.service; +package org.apache.hadoop.yarn.service.servicemonitor; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.service.ServiceContext; import org.apache.hadoop.yarn.service.component.Component; import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; import org.apache.hadoop.yarn.service.conf.YarnServiceConf; -import org.apache.slider.api.InternalKeys; import org.apache.hadoop.yarn.service.component.ComponentEvent; import org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEvent; import org.apache.hadoop.yarn.service.component.ComponentState; -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.server.servicemonitor.ProbeStatus; +import org.apache.hadoop.yarn.service.servicemonitor.probe.ProbeStatus; +import org.apache.hadoop.yarn.service.utils.SliderUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,6 +43,9 @@ import static org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEventType.BECOME_NOT_READY; import static org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEventType.BECOME_READY; import static org.apache.hadoop.yarn.service.compinstance.ComponentInstanceState.READY; +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.CONTAINER_FAILURE_WINDOW; +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.DEFAULT_READINESS_CHECK_INTERVAL; +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.READINESS_CHECK_INTERVAL; public class ServiceMonitor extends AbstractService { @@ -70,21 +72,18 @@ public void serviceInit(Configuration conf) throws Exception { @Override public void serviceStart() throws Exception { - long readinessCheckInterval = - YarnServiceConf.getLong(InternalKeys.MONITOR_INTERVAL, - InternalKeys.DEFAULT_MONITOR_INTERVAL, + long readinessCheckInterval = YarnServiceConf + .getLong(READINESS_CHECK_INTERVAL, DEFAULT_READINESS_CHECK_INTERVAL, context.application.getConfiguration(), conf); executorService .scheduleAtFixedRate(new ReadinessChecker(), readinessCheckInterval, readinessCheckInterval, TimeUnit.SECONDS); - long failureResetInterval = SliderUtils - .getTimeRange(context.application.getConfiguration(), - ResourceKeys.CONTAINER_FAILURE_WINDOW, - ResourceKeys.DEFAULT_CONTAINER_FAILURE_WINDOW_DAYS, - ResourceKeys.DEFAULT_CONTAINER_FAILURE_WINDOW_HOURS, - ResourceKeys.DEFAULT_CONTAINER_FAILURE_WINDOW_MINUTES, 0); + // Default 6 hours. + long failureResetInterval = YarnServiceConf + .getLong(CONTAINER_FAILURE_WINDOW, 21600, + context.application.getConfiguration(), conf); executorService .scheduleAtFixedRate(new ContainerFailureReset(), failureResetInterval, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/HttpProbe.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/HttpProbe.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/HttpProbe.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/HttpProbe.java index 5eba622..10c1160 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/HttpProbe.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/HttpProbe.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.slider.server.servicemonitor; +package org.apache.hadoop.yarn.service.servicemonitor.probe; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; -import org.apache.slider.common.tools.SliderUtils; +import org.apache.hadoop.yarn.service.utils.SliderUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/LogEntryBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/LogEntryBuilder.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/LogEntryBuilder.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/LogEntryBuilder.java index a1ad44f..b575d69 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/LogEntryBuilder.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/LogEntryBuilder.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.slider.server.servicemonitor; +package org.apache.hadoop.yarn.service.servicemonitor.probe; /** * Build up log entries for ease of splunk diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/MonitorKeys.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorKeys.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/MonitorKeys.java index e97ab43..f5f3d99 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorKeys.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/MonitorKeys.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.slider.server.servicemonitor; +package org.apache.hadoop.yarn.service.servicemonitor.probe; /** * Config keys for monitoring diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/MonitorUtils.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorUtils.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/MonitorUtils.java index 1e5c94c..46d1fdb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/MonitorUtils.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.slider.server.servicemonitor; +package org.apache.hadoop.yarn.service.servicemonitor.probe; -import org.apache.slider.api.resource.ReadinessCheck; +import org.apache.hadoop.yarn.service.api.records.ReadinessCheck; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/PortProbe.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/PortProbe.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/PortProbe.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/PortProbe.java index da122da..f6cf3ae 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/PortProbe.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/PortProbe.java @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.slider.server.servicemonitor; +package org.apache.hadoop.yarn.service.servicemonitor.probe; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.server.appmaster.state.RoleInstance; +import org.apache.hadoop.yarn.service.utils.SliderUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/Probe.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/Probe.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/Probe.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/Probe.java index 4809b45..b851fb7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/Probe.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/Probe.java @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.slider.server.servicemonitor; +package org.apache.hadoop.yarn.service.servicemonitor.probe; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; -import org.apache.slider.server.appmaster.state.RoleInstance; import java.io.IOException; import java.util.Map; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/ProbeStatus.java similarity index 98% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeStatus.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/ProbeStatus.java index 24668bd..7cd761c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeStatus.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/servicemonitor/probe/ProbeStatus.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.slider.server.servicemonitor; +package org.apache.hadoop.yarn.service.servicemonitor.probe; import java.io.Serializable; import java.util.Date; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelineMetricsConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelineMetricsConstants.java index 4f39921..78a7171 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelineMetricsConstants.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelineMetricsConstants.java @@ -84,9 +84,6 @@ public static final String DESCRIPTION = "DESCRIPTION"; - public static final String UNIQUE_COMPONENT_SUPPORT = - "UNIQUE_COMPONENT_SUPPORT"; - public static final String RUN_PRIVILEGED_CONTAINER = "RUN_PRIVILEGED_CONTAINER"; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelinePublisher.java index f115063..243baea 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelinePublisher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelinePublisher.java @@ -21,24 +21,19 @@ import org.apache.hadoop.metrics2.AbstractMetric; import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.api.records.ContainerState; -import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.client.api.TimelineV2Client; -import org.apache.hadoop.yarn.util.timeline.TimelineUtils; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.api.resource.ConfigFile; -import org.apache.slider.api.resource.Configuration; -import org.apache.slider.api.resource.Container; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.server.appmaster.actions.ActionStopSlider; -import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; import org.apache.hadoop.yarn.service.ServiceContext; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.RoleInstance; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.api.records.ConfigFile; +import org.apache.hadoop.yarn.service.api.records.Configuration; +import org.apache.hadoop.yarn.service.api.records.Container; +import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,6 +45,8 @@ import java.util.Map.Entry; import java.util.Set; +import static org.apache.hadoop.yarn.service.timelineservice.ServiceTimelineMetricsConstants.DIAGNOSTICS_INFO; + /** * A single service that publishes all the Timeline Entities. */ @@ -87,7 +84,8 @@ public ServiceTimelinePublisher(TimelineV2Client client) { timelineClient = client; } - public void serviceAttemptRegistered(Application application) { + public void serviceAttemptRegistered(Application application, + org.apache.hadoop.conf.Configuration systemConf) { long currentTimeMillis = application.getLaunchTime() == null ? System.currentTimeMillis() : application.getLaunchTime().getTime(); @@ -114,9 +112,12 @@ public void serviceAttemptRegistered(Application application) { // publish before configurations published putEntity(entity); - // publish application specific configurations - publishConfigurations(application.getConfiguration(), application.getId(), - ServiceTimelineEntityType.SERVICE_ATTEMPT.toString(), true); + // publish system config - YarnConfiguration + populateTimelineEntity(systemConf.iterator(), application.getId(), + ServiceTimelineEntityType.SERVICE_ATTEMPT.toString()); + // publish user conf + publishUserConf(application.getConfiguration(), application.getId(), + ServiceTimelineEntityType.SERVICE_ATTEMPT.toString()); // publish component as separate entity. publishComponents(application.getComponents()); @@ -129,12 +130,14 @@ public void serviceAttemptUpdated(Application application) { putEntity(entity); } - public void serviceAttemptUnregistered(ServiceContext context) { + public void serviceAttemptUnregistered(ServiceContext context, + String diagnostics) { TimelineEntity entity = createServiceAttemptEntity( context.attemptId.getApplicationId().toString()); Map entityInfos = new HashMap(); entityInfos.put(ServiceTimelineMetricsConstants.STATE, - FinalApplicationStatus.FAILED); + FinalApplicationStatus.ENDED); + entityInfos.put(DIAGNOSTICS_INFO, diagnostics); entity.addInfo(entityInfos); // add an event @@ -147,39 +150,6 @@ public void serviceAttemptUnregistered(ServiceContext context) { putEntity(entity); } - public void serviceAttemptUnregistered(AppState appState, - ActionStopSlider stopAction) { - long currentTimeMillis = System.currentTimeMillis(); - - TimelineEntity entity = - createServiceAttemptEntity(appState.getClusterStatus().getId()); - - // add info - Map entityInfos = new HashMap(); - entityInfos.put(ServiceTimelineMetricsConstants.EXIT_STATUS_CODE, - stopAction.getExitCode()); - entityInfos.put(ServiceTimelineMetricsConstants.STATE, - stopAction.getFinalApplicationStatus().toString()); - if (stopAction.getMessage() != null) { - entityInfos.put(ServiceTimelineMetricsConstants.EXIT_REASON, - stopAction.getMessage()); - } - if (stopAction.getEx() != null) { - entityInfos.put(ServiceTimelineMetricsConstants.DIAGNOSTICS_INFO, - stopAction.getEx().toString()); - } - entity.addInfo(entityInfos); - - // add an event - TimelineEvent startEvent = new TimelineEvent(); - startEvent - .setId(ServiceTimelineEvent.SERVICE_ATTEMPT_UNREGISTERED.toString()); - startEvent.setTimestamp(currentTimeMillis); - entity.addEvent(startEvent); - - putEntity(entity); - } - public void componentInstanceStarted(Container container, ComponentInstance instance) { @@ -210,29 +180,6 @@ public void componentInstanceStarted(Container container, putEntity(entity); } - public void componentInstanceFinished(RoleInstance instance) { - TimelineEntity entity = createComponentInstanceEntity(instance.id); - - // create info keys - Map entityInfos = new HashMap(); - entityInfos.put(ServiceTimelineMetricsConstants.EXIT_STATUS_CODE, - instance.exitCode); - entityInfos.put(ServiceTimelineMetricsConstants.DIAGNOSTICS_INFO, - instance.diagnostics); - // TODO need to change the state based on enum value. - entityInfos.put(ServiceTimelineMetricsConstants.STATE, "FINISHED"); - entity.addInfo(entityInfos); - - // add an event - TimelineEvent startEvent = new TimelineEvent(); - startEvent - .setId(ServiceTimelineEvent.COMPONENT_INSTANCE_UNREGISTERED.toString()); - startEvent.setTimestamp(System.currentTimeMillis()); - entity.addEvent(startEvent); - - putEntity(entity); - } - public void componentInstanceFinished(ComponentInstance instance, int exitCode, ContainerState state, String diagnostics) { TimelineEntity entity = createComponentInstanceEntity( @@ -242,7 +189,7 @@ public void componentInstanceFinished(ComponentInstance instance, Map entityInfos = new HashMap(); entityInfos.put(ServiceTimelineMetricsConstants.EXIT_STATUS_CODE, exitCode); - entityInfos.put(ServiceTimelineMetricsConstants.DIAGNOSTICS_INFO, diagnostics); + entityInfos.put(DIAGNOSTICS_INFO, diagnostics); entityInfos.put(ServiceTimelineMetricsConstants.STATE, state); entity.addInfo(entityInfos); @@ -302,8 +249,6 @@ private void publishComponents(List components) { entityInfos.put(ServiceTimelineMetricsConstants.LAUNCH_COMMAND, component.getLaunchCommand()); } - entityInfos.put(ServiceTimelineMetricsConstants.UNIQUE_COMPONENT_SUPPORT, - component.getUniqueComponentSupport().toString()); entityInfos.put(ServiceTimelineMetricsConstants.RUN_PRIVILEGED_CONTAINER, component.getRunPrivilegedContainer().toString()); if (component.getPlacementPolicy() != null) { @@ -315,31 +260,26 @@ private void publishComponents(List components) { putEntity(entity); // publish component specific configurations - publishConfigurations(component.getConfiguration(), component.getName(), - ServiceTimelineEntityType.COMPONENT.toString(), false); + publishUserConf(component.getConfiguration(), component.getName(), + ServiceTimelineEntityType.COMPONENT.toString()); } } - private void publishConfigurations(Configuration configuration, - String entityId, String entityType, boolean isServiceAttemptEntity) { - if (isServiceAttemptEntity) { - // publish slider-client.xml properties at service level - publishConfigurations(SliderUtils.loadSliderClientXML().iterator(), - entityId, entityType); - } - publishConfigurations(configuration.getProperties().entrySet().iterator(), + private void publishUserConf(Configuration configuration, + String entityId, String entityType) { + populateTimelineEntity(configuration.getProperties().entrySet().iterator(), entityId, entityType); - publishConfigurations(configuration.getEnv().entrySet().iterator(), + populateTimelineEntity(configuration.getEnv().entrySet().iterator(), entityId, entityType); for (ConfigFile configFile : configuration.getFiles()) { - publishConfigurations(configFile.getProps().entrySet().iterator(), + populateTimelineEntity(configFile.getProps().entrySet().iterator(), entityId, entityType); } } - private void publishConfigurations(Iterator> iterator, + private void populateTimelineEntity(Iterator> iterator, String entityId, String entityType) { int configSize = 0; TimelineEntity entity = createTimelineEntity(entityId, entityType); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ApplicationReportSerDeser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ApplicationReportSerDeser.java similarity index 94% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ApplicationReportSerDeser.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ApplicationReportSerDeser.java index a8c72ce..2607c08 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ApplicationReportSerDeser.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ApplicationReportSerDeser.java @@ -16,9 +16,8 @@ * limitations under the License. */ -package org.apache.slider.core.persist; +package org.apache.hadoop.yarn.service.utils; -import org.apache.slider.core.launch.SerializedApplicationReport; import org.codehaus.jackson.JsonGenerationException; import org.codehaus.jackson.JsonParseException; import org.codehaus.jackson.map.JsonMappingException; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientRegistryBinder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ClientRegistryBinder.java similarity index 99% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientRegistryBinder.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ClientRegistryBinder.java index da37d11..86896b2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientRegistryBinder.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ClientRegistryBinder.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.client; +package org.apache.hadoop.yarn.service.utils; import com.google.common.base.Preconditions; import org.apache.hadoop.fs.PathNotFoundException; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Comparators.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/Comparators.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Comparators.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/Comparators.java index a83901b..9f0e5d4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Comparators.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/Comparators.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.tools; +package org.apache.hadoop.yarn.service.utils; import java.io.Serializable; import java.util.Comparator; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ConfigHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ConfigHelper.java new file mode 100644 index 0000000..911fee5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ConfigHelper.java @@ -0,0 +1,157 @@ +/* + * 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.utils; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.service.exceptions.BadConfigException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.StringWriter; +import java.net.URL; +import java.util.Map; + +/** + * Methods to aid in config, both in the Configuration class and + * with other parts of setting up Slider-initated processes. + * + * Some of the methods take an argument of a map iterable for their sources; this allows + * the same method + */ +public class ConfigHelper { + private static final Logger log = LoggerFactory.getLogger(ConfigHelper.class); + + /** + * Set an entire map full of values + * + * @param config config to patch + * @param map map of data + * @param origin origin data + */ + public static void addConfigMap(Configuration config, + Map map, + String origin) throws BadConfigException { + addConfigMap(config, map.entrySet(), origin); + } + + /** + * Set an entire map full of values + * + * @param config config to patch + * @param map map of data + * @param origin origin data + */ + public static void addConfigMap(Configuration config, + Iterable> map, + String origin) throws BadConfigException { + for (Map.Entry mapEntry : map) { + String key = mapEntry.getKey(); + String value = mapEntry.getValue(); + if (value == null) { + throw new BadConfigException("Null value for property " + key); + } + config.set(key, value, origin); + } + } + + /** + * Convert to an XML string + * @param conf configuration + * @return conf + * @throws IOException + */ + public static String toXml(Configuration conf) throws IOException { + StringWriter writer = new StringWriter(); + conf.writeXml(writer); + return writer.toString(); + } + + + /** + * Register a resource as a default resource. + * Do not attempt to use this unless you understand that the + * order in which default resources are loaded affects the outcome, + * and that subclasses of Configuration often register new default + * resources + * @param resource the resource name + * @return the URL or null + */ + public static URL registerDefaultResource(String resource) { + URL resURL = getResourceUrl(resource); + if (resURL != null) { + Configuration.addDefaultResource(resource); + } + return resURL; + } + + /** + * Load a configuration from a resource on this classpath. + * If the resource is not found, an empty configuration is returned + * @param resource the resource name + * @return the loaded configuration. + */ + public static Configuration loadFromResource(String resource) { + Configuration conf = new Configuration(false); + URL resURL = getResourceUrl(resource); + if (resURL != null) { + log.debug("loaded resources from {}", resURL); + conf.addResource(resource); + } else{ + log.debug("failed to find {} on the classpath", resource); + } + return conf; + + } + + /** + * Get the URL to a resource, null if not on the CP + * @param resource resource to look for + * @return the URL or null + */ + public static URL getResourceUrl(String resource) { + return ConfigHelper.class.getClassLoader() + .getResource(resource); + } + + /** + * This goes through the keyset of one configuration and retrieves each value + * from a value source -a different or the same configuration. This triggers + * the property resolution process of the value, resolving any variables against + * in-config or inherited configurations + * @param keysource source of keys + * @param valuesource the source of values + * @return a new configuration where foreach key in keysource, get(key)==valuesource.get(key) + */ + public static Configuration resolveConfiguration( + Iterable> keysource, + Configuration valuesource) { + Configuration result = new Configuration(false); + for (Map.Entry entry : keysource) { + String key = entry.getKey(); + String value = valuesource.get(key); + Preconditions.checkState(value != null, + "no reference for \"%s\" in values", key); + result.set(key, value); + } + return result; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ConfigUtils.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigUtils.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ConfigUtils.java index 2e1615b..a969be9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ConfigUtils.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.slider.core.registry.docstore; +package org.apache.hadoop.yarn.service.utils; import org.apache.hadoop.fs.Path; -import org.apache.slider.common.tools.SliderFileSystem; +import org.apache.hadoop.yarn.service.api.records.ConfigFormat; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; import java.io.IOException; import java.util.HashMap; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/CoreFileSystem.java similarity index 67% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/CoreFileSystem.java index 43eb270..fa3b402 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/CoreFileSystem.java @@ -16,10 +16,9 @@ * limitations under the License. */ -package org.apache.slider.common.tools; +package org.apache.hadoop.yarn.service.utils; import com.google.common.base.Preconditions; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FSDataInputStream; @@ -29,32 +28,27 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; -import org.apache.hadoop.yarn.util.ConverterUtils; -import org.apache.hadoop.yarn.util.Records; +import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.service.conf.SliderExitCodes; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; -import org.apache.slider.core.exceptions.BadClusterStateException; -import org.apache.slider.core.exceptions.ErrorStrings; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.exceptions.UnknownApplicationInstanceException; -import org.apache.slider.core.persist.Filenames; +import org.apache.hadoop.yarn.service.conf.YarnServiceConstants; +import org.apache.hadoop.yarn.service.conf.YarnServiceConf; +import org.apache.hadoop.yarn.service.exceptions.BadClusterStateException; +import org.apache.hadoop.yarn.service.exceptions.ErrorStrings; +import org.apache.hadoop.yarn.service.exceptions.SliderException; +import org.apache.hadoop.yarn.util.Records; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; -import java.nio.charset.Charset; import java.util.HashMap; import java.util.Map; -import static org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys.CLUSTER_DIRECTORY_PERMISSIONS; -import static org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys.DEFAULT_CLUSTER_DIRECTORY_PERMISSIONS; - public class CoreFileSystem { private static final Logger log = LoggerFactory.getLogger(CoreFileSystem.class); @@ -87,7 +81,7 @@ public CoreFileSystem(Configuration configuration) throws IOException { */ public Path getTempPathForCluster(String clustername) { Path clusterDir = buildClusterDirPath(clustername); - return new Path(clusterDir, SliderKeys.TMP_DIR_PREFIX); + return new Path(clusterDir, YarnServiceConstants.TMP_DIR_PREFIX); } /** @@ -118,33 +112,9 @@ public String toString() { public Path buildClusterDirPath(String clustername) { Preconditions.checkNotNull(clustername); Path path = getBaseApplicationPath(); - return new Path(path, SliderKeys.CLUSTER_DIRECTORY + "/" + clustername); + return new Path(path, YarnServiceConstants.SERVICES_DIRECTORY + "/" + clustername); } - /** - * Build up the path string for app def folder -no attempt to - * create the directory is made - * - * @param clustername name of the cluster - * @return the path for persistent data - */ - public Path buildAppDefDirPath(String clustername) { - Path path = buildClusterDirPath(clustername); - return new Path(path, SliderKeys.APP_DEF_DIR); - } - - /** - * Build up the path string for addon folder -no attempt to - * create the directory is made - * - * @param clustername name of the cluster - * @return the path for persistent data - */ - public Path buildAddonDirPath(String clustername, String addonId) { - Preconditions.checkNotNull(addonId); - Path path = buildClusterDirPath(clustername); - return new Path(path, SliderKeys.ADDONS_DIR + "/" + addonId); - } /** * Build up the path string for keytab install location -no attempt to @@ -155,7 +125,7 @@ public Path buildAddonDirPath(String clustername, String addonId) { public Path buildKeytabInstallationDirPath(String keytabFolder) { Preconditions.checkNotNull(keytabFolder); Path path = getBaseApplicationPath(); - return new Path(path, SliderKeys.KEYTAB_DIR + "/" + keytabFolder); + return new Path(path, YarnServiceConstants.KEYTAB_DIR + "/" + keytabFolder); } /** @@ -171,7 +141,7 @@ public Path buildKeytabPath(String keytabDir, String keytabName, String clusterN baseKeytabDir = new Path(homePath, keytabDir); } else { baseKeytabDir = new Path(buildClusterDirPath(clusterName), - SliderKeys.KEYTAB_DIR); + YarnServiceConstants.KEYTAB_DIR); } return keytabName == null ? baseKeytabDir : new Path(baseKeytabDir, keytabName); @@ -186,7 +156,7 @@ public Path buildKeytabPath(String keytabDir, String keytabName, String clusterN public Path buildResourcePath(String resourceFolder) { Preconditions.checkNotNull(resourceFolder); Path path = getBaseApplicationPath(); - return new Path(path, SliderKeys.RESOURCE_DIR + "/" + resourceFolder); + return new Path(path, YarnServiceConstants.RESOURCE_DIR + "/" + resourceFolder); } /** @@ -199,7 +169,7 @@ public Path buildResourcePath(String dirName, String fileName) { Preconditions.checkNotNull(dirName); Preconditions.checkNotNull(fileName); Path path = getBaseApplicationPath(); - return new Path(path, SliderKeys.RESOURCE_DIR + "/" + dirName + "/" + fileName); + return new Path(path, YarnServiceConstants.RESOURCE_DIR + "/" + dirName + "/" + fileName); } /** @@ -210,6 +180,7 @@ public Path buildResourcePath(String dirName, String fileName) { * @throws IOException IO problem * @throws BadClusterStateException any cluster state problem */ + @SuppressWarnings("deprecation") public void createWithPermissions(Path dir, FsPermission clusterPerms) throws IOException, BadClusterStateException { @@ -227,25 +198,6 @@ public void createWithPermissions(Path dir, FsPermission clusterPerms) throws } /** - * Get the permissions of a path - * - * @param path path to check - * @return the permissions - * @throws IOException any IO problem (including file not found) - */ - public FsPermission getPathPermissions(Path path) throws IOException { - FileStatus status = fileSystem.getFileStatus(path); - return status.getPermission(); - } - - public FsPermission getInstanceDirectoryPermissions() { - String clusterDirPermsOct = - configuration.get(CLUSTER_DIRECTORY_PERMISSIONS, - DEFAULT_CLUSTER_DIRECTORY_PERMISSIONS); - return new FsPermission(clusterDirPermsOct); - } - - /** * Verify that the cluster directory is not present * * @param clustername name of the cluster @@ -339,7 +291,7 @@ public void verifyFileExists(Path path) throws IOException { * * @param path * absolute path to the file to check - * @returns true if and only if path exists and is a file, false for all other + * @return true if and only if path exists and is a file, false for all other * reasons including if file check throws IOException */ public boolean isFile(Path path) { @@ -359,12 +311,13 @@ public boolean isFile(Path path) { * Get the base path * * @return the base path optionally configured by - * {@link SliderXmlConfKeys#KEY_SLIDER_BASE_PATH} + * {@link YarnServiceConf#YARN_SERVICE_BASE_PATH} */ public Path getBaseApplicationPath() { - String configuredBasePath = configuration.get(SliderXmlConfKeys.KEY_SLIDER_BASE_PATH); - return configuredBasePath != null ? new Path(configuredBasePath) : - new Path(getHomeDirectory(), SliderKeys.SLIDER_BASE_DIRECTORY); + String configuredBasePath = configuration + .get(YarnServiceConf.YARN_SERVICE_BASE_PATH, + getHomeDirectory() + "/" + YarnServiceConstants.SERVICE_BASE_DIRECTORY); + return new Path(configuredBasePath); } /** @@ -373,13 +326,8 @@ public Path getBaseApplicationPath() { * @return the parent dir path of slider.tar.gz in HDFS */ public Path getDependencyPath() { - // FIXME: 3/20/17 HDP ??????????? - String parentDir = (SliderUtils.isHdp()) ? SliderKeys.SLIDER_DEPENDENCY_HDP_PARENT_DIR - + SliderKeys.SLIDER_DEPENDENCY_DIR - : SliderKeys.SLIDER_DEPENDENCY_DIR; - Path dependencyPath = new Path(String.format(parentDir, - SliderUtils.getSliderVersion())); - return dependencyPath; + String parentDir = YarnServiceConstants.DEPENDENCY_DIR; + return new Path(String.format(parentDir, VersionInfo.getVersion())); } /** @@ -391,8 +339,8 @@ public Path getDependencyTarGzip() { Path dependencyLibAmPath = getDependencyPath(); Path dependencyLibTarGzip = new Path( dependencyLibAmPath.toUri().toString(), - SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_NAME - + SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_EXT); + YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_NAME + + YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_EXT); return dependencyLibTarGzip; } @@ -400,35 +348,12 @@ public Path getHomeDirectory() { return fileSystem.getHomeDirectory(); } - public boolean maybeAddImagePath(Map localResources, - Path imagePath) throws IOException { - if (imagePath != null) { - LocalResource resource = createAmResource(imagePath, - LocalResourceType.ARCHIVE); - localResources.put(SliderKeys.LOCAL_TARBALL_INSTALL_SUBDIR, resource); - return true; - } else { - return false; - } - } - - public boolean maybeAddImagePath(Map localResources, - String imagePath) throws IOException { - - return imagePath != null && - maybeAddImagePath(localResources, new Path(imagePath)); - } - - - - /** * Create an AM resource from the * * @param destPath dest path in filesystem * @param resourceType resource type - * @return the resource set up wih application-level visibility and the - * timestamp & size set from the file stats. + * @return the local resource for AM */ public LocalResource createAmResource(Path destPath, LocalResourceType resourceType) throws IOException { FileStatus destStatus = fileSystem.getFileStatus(destPath); @@ -438,8 +363,8 @@ public LocalResource createAmResource(Path destPath, LocalResourceType resourceT // Setting to most private option amResource.setVisibility(LocalResourceVisibility.APPLICATION); // Set the resource to be copied over - amResource.setResource(ConverterUtils.getYarnUrlFromPath(fileSystem - .resolvePath(destStatus.getPath()))); + amResource.setResource( + URL.fromPath(fileSystem.resolvePath(destStatus.getPath()))); // Set timestamp and length of file so that the framework // can do basic sanity checks for the local resource // after it has been copied over to ensure it is the same @@ -530,7 +455,7 @@ public void submitTarGzipAndUpdate( Path dependencyLibTarGzip = getDependencyTarGzip(); LocalResource lc = createAmResource(dependencyLibTarGzip, LocalResourceType.ARCHIVE); - providerResources.put(SliderKeys.SLIDER_DEPENDENCY_LOCALIZED_DIR_LINK, lc); + providerResources.put(YarnServiceConstants.DEPENDENCY_LOCALIZED_DIR_LINK, lc); } public void copyLocalFileToHdfs(File localPath, @@ -542,7 +467,8 @@ public void copyLocalFileToHdfs(File localPath, fileSystem.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "000"); fileSystem.mkdirs(destPath.getParent(), fp); - log.info("Copying file {} to {}", localPath.toURI(), destPath.toUri()); + log.info("Copying file {} to {}", localPath.toURI(), + fileSystem.getScheme() + ":/" + destPath.toUri()); fileSystem.copyFromLocalFile(false, true, new Path(localPath.getPath()), destPath); @@ -580,60 +506,6 @@ public String listFSDir(Path path) throws IOException { return builder.toString(); } - /** - * List all application instances persisted for this user, giving the - * path. The instance name is the last element in the path - * @return a possibly empty map of application instance names to paths - */ - public Map listPersistentInstances() throws IOException { - FileSystem fs = getFileSystem(); - Path path = new Path(getBaseApplicationPath(), SliderKeys.CLUSTER_DIRECTORY); - log.debug("Looking for all persisted application at {}", path.toString()); - if (!fs.exists(path)) { - // special case: no instances have ever been created - return new HashMap(0); - } - FileStatus[] statuses = fs.listStatus(path); - Map instances = new HashMap(statuses.length); - - // enum the child entries - for (FileStatus status : statuses) { - if (status.isDirectory()) { - // for directories, look for an internal.json underneath - Path child = status.getPath(); - Path internalJson = new Path(child, Filenames.INTERNAL); - if (fs.exists(internalJson)) { - // success => this is an instance - instances.put(child.getName(), child); - } else { - log.info("Malformed cluster found at {}. It does not appear to be a valid persisted instance.", - child.toString()); - } - } - } - return instances; - } - - public void touch(Path path, boolean overwrite) throws IOException { - FSDataOutputStream out = null; - try { - out = fileSystem.create(path, overwrite); - } finally { - IOUtils.closeStream(out); - } - } - - public void cat(Path path, boolean overwrite, String data) throws IOException { - FSDataOutputStream out = null; - try { - out = fileSystem.create(path, overwrite); - byte[] bytes = data.getBytes(Charset.forName("UTF-8")); - out.write(bytes); - } finally { - IOUtils.closeStream(out); - } - } - public String cat(Path path) throws IOException { FileStatus status = fileSystem.getFileStatus(path); byte[] b = new byte[(int) status.getLen()]; @@ -646,55 +518,4 @@ public String cat(Path path) throws IOException { IOUtils.closeStream(in); } } - - /** - * Create a path that must exist in the cluster fs - * @param uri uri to create - * @return the path - * @throws SliderException if the path does not exist - */ - public Path createPathThatMustExist(String uri) throws - SliderException, IOException { - Preconditions.checkNotNull(uri); - Path path = new Path(uri); - verifyPathExists(path); - return path; - } - - /** - * Locate an application conf json in the FS. This includes a check to verify - * that the file is there. - * - * @param clustername name of the cluster - * @return the path to the spec. - * @throws IOException IO problems - * @throws SliderException if the path isn't there - */ - public Path locateInstanceDefinition(String clustername) throws IOException, - SliderException { - Path clusterDirectory = buildClusterDirPath(clustername); - Path appConfPath = - new Path(clusterDirectory, Filenames.APPCONF); - verifyClusterSpecExists(clustername, appConfPath); - return appConfPath; - } - - /** - * Verify that a cluster specification exists - * @param clustername name of the cluster (For errors only) - * @param clusterSpecPath cluster specification path - * @throws IOException IO problems - * @throws SliderException if the cluster specification is not present - */ - public void verifyClusterSpecExists(String clustername, Path clusterSpecPath) - throws IOException, - SliderException { - if (!fileSystem.isFile(clusterSpecPath)) { - log.debug("Missing specification file {}", clusterSpecPath); - throw UnknownApplicationInstanceException.unknownInstance( - clustername + "\n (definition not found at " + clusterSpecPath); - } - } - - } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Duration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/Duration.java similarity index 98% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Duration.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/Duration.java index e5fa424..6fadfd3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Duration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/Duration.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.tools; +package org.apache.hadoop.yarn.service.utils; import java.io.Closeable; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/JsonSerDeser.java similarity index 99% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/JsonSerDeser.java index 8fe2549..7b22e3e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/JsonSerDeser.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.core.persist; +package org.apache.hadoop.yarn.service.utils; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/security/KerberosDiags.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/KerberosDiags.java similarity index 98% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/security/KerberosDiags.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/KerberosDiags.java index 905d4b1..c0712c3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/security/KerberosDiags.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/KerberosDiags.java @@ -16,13 +16,17 @@ * limitations under the License. */ -package org.apache.hadoop.security; +package org.apache.hadoop.yarn.service.utils; import com.google.common.annotations.VisibleForTesting; import org.apache.commons.io.IOUtils; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.SaslPropertiesResolver; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.util.ExitUtil; @@ -36,7 +40,6 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.PrintStream; -import java.io.PrintWriter; import java.lang.reflect.InvocationTargetException; import java.net.InetAddress; import java.security.NoSuchAlgorithmException; @@ -162,8 +165,8 @@ public boolean execute() throws Exception { for (String env : new String[]{ "HADOOP_JAAS_DEBUG", KRB5_CCNAME, - HADOOP_USER_NAME, - HADOOP_PROXY_USER, + "HADOOP_USER_NAME", + "HADOOP_PROXY_USER", HADOOP_TOKEN_FILE_LOCATION, }) { printEnv(env); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/PatternValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/PatternValidator.java similarity index 91% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/PatternValidator.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/PatternValidator.java index 6ab9de6..6efa880 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/PatternValidator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/PatternValidator.java @@ -16,9 +16,7 @@ * limitations under the License. */ -package org.apache.slider.server.services.utility; - -import org.apache.slider.server.appmaster.web.rest.RestPaths; +package org.apache.hadoop.yarn.service.utils; import java.util.regex.Pattern; @@ -39,7 +37,6 @@ public PatternValidator(String pattern) { /** * Validate the name -restricting it to the set defined in - * {@link RestPaths#PUBLISHED_CONFIGURATION_REGEXP} * @param name name to validate * @throws IllegalArgumentException if not a valid name */ diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/PortScanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/PortScanner.java similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/PortScanner.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/PortScanner.java index 235d3da..2dbf37f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/PortScanner.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/PortScanner.java @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.slider.common.tools; +package org.apache.hadoop.yarn.service.utils; import org.apache.hadoop.yarn.service.conf.SliderExitCodes; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.core.exceptions.SliderException; +import org.apache.hadoop.yarn.service.exceptions.BadConfigException; +import org.apache.hadoop.yarn.service.exceptions.SliderException; import java.io.IOException; import java.util.ArrayList; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/PublishedConfiguration.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfiguration.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/PublishedConfiguration.java index 50b522f..9d00b3c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/PublishedConfiguration.java @@ -16,11 +16,11 @@ * limitations under the License. */ -package org.apache.slider.core.registry.docstore; +package org.apache.hadoop.yarn.service.utils; import org.apache.hadoop.conf.Configuration; -import org.apache.slider.common.tools.ConfigHelper; -import org.apache.slider.core.exceptions.BadConfigException; +import org.apache.hadoop.yarn.service.api.records.ConfigFormat; +import org.apache.hadoop.yarn.service.exceptions.BadConfigException; import org.codehaus.jackson.annotate.JsonIgnoreProperties; import org.codehaus.jackson.map.ObjectMapper; import org.codehaus.jackson.map.SerializationConfig; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/PublishedConfigurationOutputter.java similarity index 98% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/PublishedConfigurationOutputter.java index 4ec513c..88ecf2c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/PublishedConfigurationOutputter.java @@ -16,14 +16,14 @@ * limitations under the License. */ -package org.apache.slider.core.registry.docstore; +package org.apache.hadoop.yarn.service.utils; import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; -import org.apache.slider.common.tools.ConfigHelper; +import org.apache.hadoop.yarn.service.api.records.ConfigFormat; import org.yaml.snakeyaml.DumperOptions; import org.yaml.snakeyaml.DumperOptions.FlowStyle; import org.yaml.snakeyaml.Yaml; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/SerializedApplicationReport.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/SerializedApplicationReport.java index 8e0ef5a..405f690 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/SerializedApplicationReport.java @@ -16,12 +16,12 @@ * limitations under the License. */ -package org.apache.slider.core.launch; +package org.apache.hadoop.yarn.service.utils; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.slider.core.persist.ApplicationReportSerDeser; +import org.apache.hadoop.yarn.service.utils.ApplicationReportSerDeser; import org.codehaus.jackson.annotate.JsonIgnoreProperties; import org.codehaus.jackson.map.annotate.JsonSerialize; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java index c87c3b4..21cb049 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java @@ -25,20 +25,16 @@ import org.apache.hadoop.registry.client.api.RegistryConstants; import org.apache.hadoop.registry.client.binding.RegistryUtils; import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Artifact; -import org.apache.slider.api.resource.Component; -import org.apache.slider.api.resource.Configuration; -import org.apache.slider.api.resource.Resource; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.persist.JsonSerDeser; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.api.records.Artifact; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.api.records.Configuration; +import org.apache.hadoop.yarn.service.api.records.Resource; import org.apache.hadoop.yarn.service.provider.AbstractClientProvider; import org.apache.hadoop.yarn.service.provider.ProviderFactory; -import org.apache.slider.server.servicemonitor.MonitorUtils; -import org.apache.slider.server.services.utility.PatternValidator; -import org.apache.slider.util.RestApiConstants; -import org.apache.slider.util.RestApiErrorMessages; +import org.apache.hadoop.yarn.service.servicemonitor.probe.MonitorUtils; +import org.apache.hadoop.yarn.service.conf.RestApiConstants; +import org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages; import org.codehaus.jackson.map.PropertyNamingStrategy; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -127,9 +123,7 @@ public static void validateAndResolveApplication(Application application, List componentsToAdd = new ArrayList<>(); for (Component comp : application.getComponents()) { int maxCompLength = RegistryConstants.MAX_FQDN_LABEL_LENGTH; - if (comp.getUniqueComponentSupport()) { - maxCompLength = maxCompLength - Long.toString(Long.MAX_VALUE).length(); - } + maxCompLength = maxCompLength - Long.toString(Long.MAX_VALUE).length(); if (dnsEnabled && comp.getName().length() > maxCompLength) { throw new IllegalArgumentException(String.format(RestApiErrorMessages .ERROR_COMPONENT_NAME_INVALID, maxCompLength, comp.getName())); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceRegistryUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceRegistryUtils.java new file mode 100644 index 0000000..7440b11 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceRegistryUtils.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.utils; + +import org.apache.hadoop.registry.client.binding.RegistryUtils; +import org.apache.hadoop.yarn.service.conf.YarnServiceConstants; + + +public class ServiceRegistryUtils { + + /** + * Base path for services + */ + public static final String ZK_SERVICES = "services"; + + /** + * Base path for all Slider references + */ + public static final String ZK_SLIDER = "slider"; + public static final String ZK_USERS = "users"; + public static final String SVC_SLIDER = "/" + ZK_SERVICES + "/" + ZK_SLIDER; + public static final String SVC_SLIDER_USERS = SVC_SLIDER + "/" + ZK_USERS; + + /** + * Get the registry path for an instance under the user's home node + * @param instanceName application instance + * @return a path to the registry location for this application instance. + */ + public static String registryPathForInstance(String instanceName) { + return RegistryUtils.servicePath( + RegistryUtils.currentUser(), YarnServiceConstants.APP_TYPE, instanceName + ); + } + + /** + * Build the path to a cluster; exists once the cluster has come up. + * Even before that, a ZK watcher could wait for it. + * @param username user + * @param clustername name of the cluster + * @return a strin + */ + public static String mkClusterPath(String username, String clustername) { + return mkSliderUserPath(username) + "/" + clustername; + } + + /** + * Build the path to a cluster; exists once the cluster has come up. + * Even before that, a ZK watcher could wait for it. + * @param username user + * @return a string + */ + public static String mkSliderUserPath(String username) { + return SVC_SLIDER_USERS + "/" + username; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderFileSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/SliderFileSystem.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderFileSystem.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/SliderFileSystem.java index 40b07bd..d6d664e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderFileSystem.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/SliderFileSystem.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.tools; +package org.apache.hadoop.yarn.service.utils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/SliderUtils.java similarity index 65% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/SliderUtils.java index fc57c82..415392a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/SliderUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.slider.common.tools; +package org.apache.hadoop.yarn.service.utils; import com.google.common.base.Preconditions; import org.apache.commons.compress.archivers.tar.TarArchiveEntry; @@ -37,30 +37,21 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.Shell; -import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.LocalResource; -import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.client.api.AMRMClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.slider.Slider; -import org.apache.slider.api.RoleKeys; -import org.apache.slider.api.types.ContainerInformation; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; import org.apache.hadoop.yarn.service.client.params.Arguments; import org.apache.hadoop.yarn.service.client.params.SliderActions; -import org.apache.slider.core.exceptions.BadClusterStateException; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.launch.ClasspathConstructor; -import org.apache.slider.core.main.LauncherExitCodes; -import org.apache.slider.server.services.utility.PatternValidator; -import org.apache.slider.server.services.workflow.ForkedProcessService; +import org.apache.hadoop.yarn.service.conf.YarnServiceConstants; +import org.apache.hadoop.yarn.service.containerlaunch.ClasspathConstructor; +import org.apache.hadoop.yarn.service.exceptions.BadClusterStateException; +import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; +import org.apache.hadoop.yarn.service.exceptions.BadConfigException; +import org.apache.hadoop.yarn.service.exceptions.LauncherExitCodes; +import org.apache.hadoop.yarn.service.exceptions.SliderException; import org.apache.zookeeper.server.util.KerberosUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,15 +63,10 @@ import java.io.FileOutputStream; import java.io.FilenameFilter; import java.io.IOException; -import java.io.InputStream; -import java.io.PrintWriter; import java.io.Serializable; -import java.io.StringWriter; import java.net.InetSocketAddress; import java.net.ServerSocket; import java.net.Socket; -import java.net.URI; -import java.net.URISyntaxException; import java.net.URL; import java.net.URLDecoder; import java.text.DateFormat; @@ -91,27 +77,19 @@ import java.util.Collections; import java.util.Comparator; import java.util.Date; -import java.util.EnumSet; import java.util.Enumeration; import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Properties; import java.util.Set; import java.util.TimeZone; -import java.util.TimerTask; -import java.util.TreeMap; -import java.util.TreeSet; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; import java.util.zip.GZIPOutputStream; import java.util.zip.ZipEntry; import java.util.zip.ZipOutputStream; -import static org.apache.hadoop.yarn.service.conf.SliderKeys.COMPONENT_SEPARATOR; - /** * These are slider-specific Util methods */ @@ -423,6 +401,7 @@ public static boolean oldIsClusternameValid(String name) { * @param permission permission for the dest directory; null means "default" * @return # of files copies */ + @SuppressWarnings("deprecation") public static int copyDirectory(Configuration conf, Path srcDirPath, Path destDirPath, @@ -478,6 +457,7 @@ public static int copyDirectory(Configuration conf, * @param srcFile src file * @param destFile dest file */ + @SuppressWarnings("deprecation") public static void copy(Configuration conf, Path srcFile, Path destFile) throws @@ -496,61 +476,6 @@ public static void copy(Configuration conf, FileUtil.copy(srcFS, srcFile, destFS, destFile, false, true, conf); } - public static String stringify(Throwable t) { - StringWriter sw = new StringWriter(); - sw.append(t.toString()).append('\n'); - t.printStackTrace(new PrintWriter(sw)); - return sw.toString(); - } - - /** - * Extract the first line of a multi-line string. This is typically used to - * prune the stack trace appended to the end of exception messages returned by - * YARN in AMRMClientAsync callbacks. - * - * @param msg - * message string (most likely multi-lines) - * @return the first line of a multi-line string or the original string if it - * is a null, empty or single-line - */ - public static String extractFirstLine(String msg) { - if (StringUtils.isNotBlank(msg)) { - int newlineIndex = msg.indexOf(System.lineSeparator()); - if (newlineIndex != -1) { - msg = msg.substring(0, newlineIndex); - } - } - return msg; - } - - /** - * Create a configuration with Slider-specific tuning. - * This is done rather than doing custom configs. - * @return the config - */ - public static YarnConfiguration createConfiguration() { - YarnConfiguration conf = new YarnConfiguration(); - patchConfiguration(conf); - return conf; - } - - /** - * Take an existing conf and patch it for Slider's needs. Useful - * in Service.init & RunService methods where a shared config is being - * passed in - * @param conf configuration - * @return the patched configuration - */ - public static Configuration patchConfiguration(Configuration conf) { - - //if the fallback option is NOT set, enable it. - //if it is explicitly set to anything -leave alone - if (conf.get(SliderXmlConfKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH) == null) { - conf.set(SliderXmlConfKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH, "true"); - } - return conf; - } - /** * Take a collection, return a list containing the string value of every * element in the collection. @@ -714,72 +639,6 @@ public static String appReportToString(ApplicationReport r, } /** - * Convert the instance details of an application to a string - * @param name instance name - * @param report the application report - * @param verbose verbose output - * @return a string - */ - public static String instanceDetailsToString(String name, - ApplicationReport report, - List containers, - String version, - Set components, - boolean verbose) { - // format strings - String staticf = "%-30s"; - String reportedf = staticf + " %10s %-42s"; - String livef = reportedf + " %s"; - StringBuilder builder = new StringBuilder(200); - if (report == null) { - builder.append(String.format(staticf, name)); - } else { - // there's a report to look at - String appId = report.getApplicationId().toString(); - String state = report.getYarnApplicationState().toString(); - if (report.getYarnApplicationState() == YarnApplicationState.RUNNING) { - // running: there's a URL - builder.append( - String.format(livef, name, state, appId, report.getTrackingUrl())); - } else { - builder.append(String.format(reportedf, name, state, appId)); - } - if (verbose) { - builder.append('\n'); - builder.append(SliderUtils.appReportToString(report, "\n ")); - } - if (containers != null) { - builder.append('\n'); - builder.append(SliderUtils.containersToString(containers, version, - components)); - } - } - - builder.append('\n'); - return builder.toString(); - } - - public static String containersToString( - List containers, String version, - Set components) { - String containerf = " %-28s %30s %45s %s%n"; - StringBuilder builder = new StringBuilder(512); - builder.append("Containers:%n"); - builder.append(String.format(" %-28s %30s %45s %s%n", "Component Name", - "App Version", "Container Id", "Container Info/Logs")); - for (ContainerInformation container : containers) { - if (filter(container.appVersion, version) - || filter(container.component, components)) { - continue; - } - builder.append(String.format(containerf, container.component, - container.appVersion, container.containerId, container.host - + SliderKeys.YARN_CONTAINER_PATH + container.containerId)); - } - return builder.toString(); - } - - /** * Filter a string value given a single filter * * @param value @@ -856,54 +715,6 @@ public static void sortApplicationReport(List instances) { } /** - * Built a (sorted) map of application reports, mapped to the instance name - * The list is sorted, and the addition process does not add a report - * if there is already one that exists. If the list handed in is sorted, - * those that are listed first form the entries returned - * @param instances list of intances - * @param minState minimum YARN state to be included - * @param maxState maximum YARN state to be included - * @return all reports in the list whose state >= minimum and <= maximum - */ - public static Map buildApplicationReportMap( - List instances, - YarnApplicationState minState, YarnApplicationState maxState) { - TreeMap map = new TreeMap<>(); - for (ApplicationReport report : instances) { - YarnApplicationState state = report.getYarnApplicationState(); - if (state.ordinal() >= minState.ordinal() && - state.ordinal() <= maxState.ordinal() && - map.get(report.getName()) == null) { - map.put(report.getName(), report); - } - } - return map; - } - - /** - * Take a map and produce a sorted equivalent - * @param source source map - * @return a map whose iterator returns the string-sorted ordering of entries - */ - public static Map sortedMap(Map source) { - Map out = new TreeMap<>(source); - return out; - } - - /** - * Convert a properties instance to a string map. - * @param properties source property object - * @return a string map - */ - public static Map toMap(Properties properties) { - Map out = new HashMap<>(properties.size()); - for (Map.Entry entry : properties.entrySet()) { - out.put(entry.getKey().toString(), entry.getValue().toString()); - } - return out; - } - - /** * Merge in one map to another -all entries in the second map are * merged into the first -overwriting any duplicate keys. * @param first first map -the updated one. @@ -967,38 +778,6 @@ public static void sortApplicationReport(List instances) { } /** - * Merge string maps excluding prefixes - * @param first first map - * @param second second map - * @param prefixes prefixes to ignore - * @return 'first' merged with the second - */ - public static Map mergeMapsIgnoreDuplicateKeysAndPrefixes( - Map first, Map second, - List prefixes) { - Preconditions.checkArgument(first != null, "Null 'first' value"); - Preconditions.checkArgument(second != null, "Null 'second' value"); - Preconditions.checkArgument(prefixes != null, "Null 'prefixes' value"); - for (Map.Entry entry : second.entrySet()) { - String key = entry.getKey(); - boolean hasPrefix = false; - for (String prefix : prefixes) { - if (key.startsWith(prefix)) { - hasPrefix = true; - break; - } - } - if (hasPrefix) { - continue; - } - if (!first.containsKey(key)) { - first.put(key, entry.getValue()); - } - } - return first; - } - - /** * Convert a map to a multi-line string for printing * @param map map to stringify * @return a string representation of the map @@ -1016,27 +795,6 @@ public static String stringifyMap(Map map) { } /** - * Get the int value of a role - * @param roleMap map of role key->val entries - * @param key key the key to look for - * @param defVal default value to use if the key is not in the map - * @param min min value or -1 for do not check - * @param max max value or -1 for do not check - * @return the int value the integer value - * @throws BadConfigException if the value could not be parsed - */ - public static int getIntValue(Map roleMap, - String key, - int defVal, - int min, - int max - ) throws BadConfigException { - String valS = roleMap.get(key); - return parseAndValidate(key, valS, defVal, min, max); - - } - - /** * Parse an int value, replacing it with defval if undefined; * @param errorKey key to use in exceptions * @param defVal default value to use if the key is not in the map @@ -1170,24 +928,6 @@ public static String stringify(org.apache.hadoop.yarn.api.records.URL url) { return builder.toString(); } - public static int findFreePort(int start, int limit) { - if (start == 0) { - //bail out if the default is "dont care" - return 0; - } - int found = 0; - int port = start; - int finish = start + limit; - while (found == 0 && port < finish) { - if (isPortAvailable(port)) { - found = port; - } else { - port++; - } - } - return found; - } - /** * Get a random open port * @return true if the port was available for listening on @@ -1220,23 +960,10 @@ public static boolean isPortAvailable(int port) { } } - /** - * Build the environment map from a role option map, finding all entries - * beginning with "env.", adding them to a map of (prefix-removed) - * env vars - * @param roleOpts role options. This can be null, meaning the - * role is undefined - * @return a possibly empty map of environment variables. - */ - public static Map buildEnvMap(Map roleOpts) { - return buildEnvMap(roleOpts, null); - } - - // Build env map: key -> value; // value will be replaced by the corresponding value in tokenMap, if any. public static Map buildEnvMap( - org.apache.slider.api.resource.Configuration conf, + org.apache.hadoop.yarn.service.api.records.Configuration conf, Map tokenMap) { if (tokenMap == null) { return conf.getEnv(); @@ -1254,31 +981,6 @@ public static boolean isPortAvailable(int port) { return env; } - - public static Map buildEnvMap(Map roleOpts, - Map tokenMap) { - Map env = new HashMap<>(); - if (roleOpts != null) { - for (Map.Entry entry : roleOpts.entrySet()) { - String key = entry.getKey(); - if (key.startsWith(RoleKeys.ENV_PREFIX)) { - String envName = key.substring(RoleKeys.ENV_PREFIX.length()); - if (!envName.isEmpty()) { - String value = entry.getValue(); - if (tokenMap != null) { - for (Map.Entry token : tokenMap.entrySet()) { - value = value.replaceAll(Pattern.quote(token.getKey()), - token.getValue()); - } - } - env.put(envName, value); - } - } - } - } - return env; - } - /** * Apply a set of command line options to a cluster role map * @param clusterRoleMap cluster role map to merge onto @@ -1410,7 +1112,7 @@ public static boolean initProcessSecurity(Configuration conf) throws } SliderUtils.verifyPrincipalSet(conf, YarnConfiguration.RM_PRINCIPAL); - SliderUtils.verifyPrincipalSet(conf, SliderXmlConfKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY); + SliderUtils.verifyPrincipalSet(conf, "dfs.namenode.kerberos.principal"); return true; } @@ -1438,7 +1140,7 @@ public static String getLibDir() { } public static String[] getLibDirs() { - String libDirStr = System.getProperty(SliderKeys.PROPERTY_LIB_DIR); + String libDirStr = System.getProperty(YarnServiceConstants.PROPERTY_LIB_DIR); if (isUnset(libDirStr)) { return ArrayUtils.EMPTY_STRING_ARRAY; } @@ -1478,7 +1180,6 @@ public static LocalResource putJar(Map providerResources, * @param sliderFileSystem remote fs * @param libDir lib directory * @param srcPath copy jars from - * @throws IOException, SliderException trouble copying to HDFS */ public static void putAllJars(Map providerResources, SliderFileSystem sliderFileSystem, @@ -1516,15 +1217,14 @@ public boolean accept(File dir, String name) { * Submit the AM tar.gz containing all dependencies and map it * @param providerResources provider map to build up * @param sliderFileSystem remote fs - * @throws IOException, SliderException trouble copying to HDFS */ public static void putAmTarGzipAndUpdate( Map providerResources, SliderFileSystem sliderFileSystem ) throws IOException, SliderException { log.info("Loading all dependencies from {}{}", - SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_NAME, - SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_EXT); + YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_NAME, + YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_EXT); sliderFileSystem.submitTarGzipAndUpdate(providerResources); } @@ -1570,24 +1270,6 @@ public static Path createLocalPath(File file) { return new Path(file.toURI()); } - /** - * Get the current user -relays to - * {@link UserGroupInformation#getCurrentUser()} - * with any Slider-specific post processing and exception handling - * @return user info - * @throws IOException on a failure to get the credentials - */ - public static UserGroupInformation getCurrentUser() throws IOException { - - try { - UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); - return currentUser; - } catch (IOException e) { - log.info("Failed to get user info", e); - throw e; - } - } - public static String getKerberosRealm() { try { return KerberosUtil.getDefaultRealm(); @@ -1599,37 +1281,6 @@ public static String getKerberosRealm() { } /** - * Register the client resource in - * {@link SliderKeys#SLIDER_CLIENT_XML} - * for Configuration instances. - * - * @return true if the resource could be loaded - */ - public static URL registerClientResource() { - return ConfigHelper.registerDefaultResource(SliderKeys.SLIDER_CLIENT_XML); - } - - /** - * Attempt to load the slider client resource. If the - * resource is not on the CP an empty config is returned. - * @return a config - */ - public static Configuration loadSliderClientXML() { - return ConfigHelper.loadFromResource(SliderKeys.SLIDER_CLIENT_XML); - } - - /** - * Convert a char sequence to a string. - * This ensures that comparisons work - * @param charSequence source - * @return the string equivalent - */ - public static String sequenceToString(CharSequence charSequence) { - StringBuilder stringBuilder = new StringBuilder(charSequence); - return stringBuilder.toString(); - } - - /** * Build up the classpath for execution * -behaves very differently on a mini test cluster vs a production * production one. @@ -1647,7 +1298,7 @@ public static ClasspathConstructor buildClasspath(String sliderConfDir, boolean usingMiniMRCluster) { ClasspathConstructor classpath = new ClasspathConstructor(); - classpath.append(SliderKeys.LOG4J_SERVER_PROP_FILENAME); + classpath.append(YarnServiceConstants.YARN_SERVICE_LOG4J_FILENAME); // add the runtime classpath needed for tests to work if (usingMiniMRCluster) { @@ -1660,7 +1311,7 @@ public static ClasspathConstructor buildClasspath(String sliderConfDir, } classpath.addLibDir(libdir); if (sliderFileSystem.isFile(sliderFileSystem.getDependencyTarGzip())) { - classpath.addLibDir(SliderKeys.SLIDER_DEPENDENCY_LOCALIZED_DIR_LINK); + classpath.addLibDir(YarnServiceConstants.DEPENDENCY_LOCALIZED_DIR_LINK); } else { log.info( "For faster submission of apps, upload dependencies using cmd {} {}", @@ -1765,47 +1416,6 @@ public static File verifyKeytabExists(Configuration siteConf, } /** - * Convert an epoch time to a GMT time. This - * uses the deprecated Date.toString() operation, - * so is in one place to reduce the number of deprecation warnings. - * @param time timestamp - * @return string value as ISO-9601 - */ - @SuppressWarnings({"CallToDateToString", "deprecation"}) - public static String toGMTString(long time) { - return new Date(time).toGMTString(); - } - - /** - * Add the cluster build information; this will include Hadoop details too - * @param info cluster info - * @param prefix prefix for the build info - */ - public static void addBuildInfo(Map info, String prefix) { - - Properties props = SliderVersionInfo.loadVersionProperties(); - info.put(prefix + "." + SliderVersionInfo.APP_BUILD_INFO, props.getProperty( - SliderVersionInfo.APP_BUILD_INFO)); - info.put(prefix + "." + SliderVersionInfo.HADOOP_BUILD_INFO, - props.getProperty(SliderVersionInfo.HADOOP_BUILD_INFO)); - - info.put(prefix + "." + SliderVersionInfo.HADOOP_DEPLOYED_INFO, - VersionInfo.getBranch() + " @" + VersionInfo.getSrcChecksum()); - } - - public static String propertiesToString(Properties props) { - TreeSet keys = new TreeSet<>(props.stringPropertyNames()); - StringBuilder builder = new StringBuilder(); - for (String key : keys) { - builder.append(key) - .append("=") - .append(props.getProperty(key)) - .append("\n"); - } - return builder.toString(); - } - - /** * Add a subpath to an existing URL. This extends * the path, inserting a / between all entries * if needed. @@ -1827,21 +1437,6 @@ public static String appendToURL(String base, String path) { } /** - * Append a list of paths, inserting "/" signs as appropriate - * @param base base path/URL - * @param paths subpaths - * @return base+"/"+paths[0]+"/"+paths[1]... - */ - public static String appendToURL(String base, String... paths) { - String result = base; - for (String path : paths) { - result = appendToURL(result, path); - } - return result; - } - - - /** * Truncate the given string to a maximum length provided * with a pad (...) added to the end if expected size if more than 10. * @param toTruncate string to truncate; may be null @@ -1862,62 +1457,6 @@ public static String truncate(String toTruncate, int maxSize) { } /** - * Get a string node label value from a node report - * @param report node report - * @return a single trimmed label or "" - */ - public static String extractNodeLabel(NodeReport report) { - Set newlabels = report.getNodeLabels(); - if (newlabels != null && !newlabels.isEmpty()) { - return newlabels.iterator().next().trim(); - } else { - return ""; - } - } - - /** - * Callable for async/scheduled halt - */ - public static class DelayedHalt extends TimerTask { - private final int status; - private final String text; - - public DelayedHalt(int status, String text) { - this.status = status; - this.text = text; - } - - @Override - public void run() { - try { - ExitUtil.halt(status, text); - //this should never be reached - } catch (ExitUtil.HaltException e) { - log.info("Halt failed"); - } - } - } - - /** - * A compareTo function that converts the result of a long - * comparision into the integer that Comparable - * expects. - * @param left left side - * @param right right side - * @return -1, 0, 1 depending on the diff - */ - public static int compareTo(long left, long right) { - long diff = left - right; - if (diff < 0) { - return -1; - } - if (diff > 0) { - return 1; - } - return 0; - } - - /** * Given a source folder create zipped file * * @param srcFolder @@ -1984,45 +1523,6 @@ public static void tarGzipFolder(String[] libDirs, File tarGzipFile, } } - /** - * Retrieve the HDP version if it is an HDP cluster, or null otherwise. It - * first checks if system property HDP_VERSION is defined. If not it checks if - * system env HDP_VERSION is defined. - * - * @return HDP version (if defined) or null otherwise - */ - public static String getHdpVersion() { - String hdpVersion = System - .getProperty(SliderKeys.HDP_VERSION_PROP_NAME); - if (StringUtils.isEmpty(hdpVersion)) { - hdpVersion = System.getenv(SliderKeys.HDP_VERSION_PROP_NAME); - } - return hdpVersion; - } - - /** - * Query to find if it is an HDP cluster - * - * @return true if this is invoked in an HDP cluster or false otherwise - */ - public static boolean isHdp() { - return StringUtils.isNotEmpty(getHdpVersion()); - } - - /** - * Retrieve the version of the current Slider install - * - * @return the version string of the Slider release - */ - public static String getSliderVersion() { - if (isHdp()) { - return getHdpVersion(); - } else { - Properties props = SliderVersionInfo.loadVersionProperties(); - return props.getProperty(SliderVersionInfo.APP_VERSION); - } - } - private static void generateFileList(List fileList, File node, File rootFolder, Boolean relative) { generateFileList(fileList, node, rootFolder, relative, null); @@ -2053,23 +1553,6 @@ private static void generateFileList(List fileList, File node, } /** - * This wraps ApplicationReports and generates a string version - * iff the toString() operator is invoked - */ - public static class OnDemandReportStringifier { - private final ApplicationReport report; - - public OnDemandReportStringifier(ApplicationReport report) { - this.report = report; - } - - @Override - public String toString() { - return appReportToString(report, "\n"); - } - } - - /** * Check for any needed libraries being present. On Unix none are needed; * on windows they must be present * @return true if all is well @@ -2116,26 +1599,6 @@ public static void maybeVerifyWinUtilsValid() throws } } - public static void verifyIsFile(String program, File exe) throws - FileNotFoundException { - if (!exe.isFile()) { - throw new FileNotFoundException(program - + " at " + exe - + " is not a file"); - - } - } - - public static void verifyFileSize(String program, - File exe, - int minFileSize) throws FileNotFoundException { - if (exe.length() < minFileSize) { - throw new FileNotFoundException(program - + " at " + exe - + " is too short to be an executable"); - } - } - /** * Write bytes to a file * @param outfile output file @@ -2160,217 +1623,6 @@ public static void write(File outfile, byte[] data) } /** - * Execute a command for a test operation - * @param name name in error - * @param status status code expected - * @param timeoutMillis timeout in millis for process to finish - * @param logger - * @param outputString optional string to grep for (must not span a line) - * @param commands commands @return the process - * @throws IOException on any failure. - */ - public static ForkedProcessService execCommand(String name, - int status, - long timeoutMillis, - Logger logger, - String outputString, - String... commands) throws IOException, SliderException { - Preconditions.checkArgument(isSet(name), "no name"); - Preconditions.checkArgument(commands.length > 0, "no commands"); - Preconditions.checkArgument(isSet(commands[0]), "empty command"); - - ForkedProcessService process; - - - process = new ForkedProcessService( - name, - new HashMap(), - Arrays.asList(commands)); - process.setProcessLog(logger); - process.init(new Configuration()); - String errorText = null; - process.start(); - try { - if (!process.waitForServiceToStop(timeoutMillis)) { - throw new TimeoutException( - "Process did not stop in " + timeoutMillis + "mS"); - } - int exitCode = process.getExitCode(); - List recentOutput = process.getRecentOutput(); - if (status != exitCode) { - // error condition - errorText = "Expected exit code={" + status + "}, " - + "actual exit code={" + exitCode + "}"; - } else { - if (isSet(outputString)) { - boolean found = false; - for (String line : recentOutput) { - if (line.contains(outputString)) { - found = true; - break; - } - } - if (!found) { - errorText = "Did not find \"" + outputString + "\"" - + " in output"; - } - } - } - if (errorText == null) { - return process; - } - - } catch (TimeoutException e) { - errorText = e.toString(); - } - // error text: non null ==> operation failed - log.warn(errorText); - List recentOutput = process.getRecentOutput(); - for (String line : recentOutput) { - log.info(line); - } - throw new SliderException(LauncherExitCodes.EXIT_OTHER_FAILURE, - "Process %s failed: %s", name, errorText); - - } - - - /** - * Validate the slider client-side execution environment. - * This looks for everything felt to be critical for execution, including - * native binaries and other essential dependencies. - * @param logger logger to log to on normal execution - * @throws IOException on IO failures - * @throws SliderException on validation failures - */ - public static void validateSliderClientEnvironment(Logger logger) throws - IOException, - SliderException { - maybeVerifyWinUtilsValid(); - } - - /** - * Validate the slider server-side execution environment. - * This looks for everything felt to be critical for execution, including - * native binaries and other essential dependencies. - * @param logger logger to log to on normal execution - * @param dependencyChecks flag to indicate checks for agent dependencies - * @throws IOException on IO failures - * @throws SliderException on validation failures - */ - public static void validateSliderServerEnvironment(Logger logger, - boolean dependencyChecks) throws - IOException, - SliderException { - maybeVerifyWinUtilsValid(); - if (dependencyChecks) { - validatePythonEnv(logger); - validateOpenSSLEnv(logger); - } - } - - public static void validateOpenSSLEnv(Logger logger) throws - IOException, - SliderException { - execCommand(OPENSSL, 0, 5000, logger, "OpenSSL", OPENSSL, "version"); - } - - public static void validatePythonEnv(Logger logger) throws - IOException, - SliderException { - execCommand(PYTHON, 0, 5000, logger, "Python", PYTHON, "-V"); - } - - /** - * return the path to the currently running slider command - * - * @throws NullPointerException - * - If the pathname argument is null - * @throws SecurityException - * - if a security manager exists and its checkPermission method - * doesn't allow getting the ProtectionDomain - */ - public static String getCurrentCommandPath() { - File f = new File(Slider.class.getProtectionDomain().getCodeSource() - .getLocation().getPath()); - return f.getAbsolutePath(); - } - - /** - * return the path to the slider-client.xml used by the current running - * slider command - * - * @throws SecurityException - * - if a security manager exists and its checkPermission method - * denies access to the class loader for the class - */ - public static String getClientConfigPath() { - URL path = ConfigHelper.class.getClassLoader().getResource( - SliderKeys.SLIDER_CLIENT_XML); - Preconditions.checkNotNull(path, "Failed to locate resource " + SliderKeys.SLIDER_CLIENT_XML); - return path.toString(); - } - - /** - * validate if slider-client.xml under the path can be opened - * - * @throws IOException - * : the file can't be found or open - */ - public static void validateClientConfigFile() throws IOException { - URL resURL = SliderVersionInfo.class.getClassLoader().getResource( - SliderKeys.SLIDER_CLIENT_XML); - if (resURL == null) { - throw new IOException( - "slider-client.xml doesn't exist on the path: " - + getClientConfigPath()); - } - - try { - InputStream inStream = resURL.openStream(); - if (inStream == null) { - throw new IOException("slider-client.xml can't be opened"); - } - } catch (IOException e) { - throw new IOException("slider-client.xml can't be opened: " - + e.toString()); - } - } - - /** - * validate if a file on HDFS can be open - * - * @throws IOException the file can't be found or opened - * @throws URISyntaxException - */ - public static void validateHDFSFile(SliderFileSystem sliderFileSystem, - String pathStr) - throws IOException, URISyntaxException { - try(InputStream inputStream = - sliderFileSystem.getFileSystem().open(new Path(new URI(pathStr)))) { - if (inputStream == null) { - throw new IOException("HDFS file " + pathStr + " can't be opened"); - } - } - } - - /** - * return the version and path of the JDK invoking the current running - * slider command - * - * @throws SecurityException - * - if a security manager exists and its checkPropertyAccess - * method doesn't allow access to the specified system property. - */ - public static String getJDKInfo() { - String version = System.getProperty("java.version"); - String javaHome = System.getProperty("java.home"); - return - "The version of the JDK invoking the current running slider command: " - + version + "; The path to it is: " + javaHome; - } - - /** * Compare the times of two applications: most recent app comes first * Specifically: the one whose start time value is greater. */ @@ -2428,51 +1680,6 @@ public static int compareTwoLongsReverse(long x, long y) { return (x < y) ? 1 : ((x == y) ? 0 : -1); } - public static String getSystemEnv(String property) { - return System.getenv(property); - } - - public static Map getSystemEnv() { - return System.getenv(); - } - - public static String requestToString(AMRMClient.ContainerRequest request) { - Preconditions.checkArgument(request != null, "Null request"); - StringBuilder buffer = new StringBuilder(request.toString()); - buffer.append("; "); - buffer.append("relaxLocality=").append(request.getRelaxLocality()).append("; "); - String labels = request.getNodeLabelExpression(); - if (labels != null) { - buffer.append("nodeLabels=").append(labels).append("; "); - } - List nodes = request.getNodes(); - if (nodes != null) { - buffer.append("Nodes = [ "); - int size = nodes.size(); - for (int i = 0; i < Math.min(NODE_LIST_LIMIT, size); i++) { - buffer.append(nodes.get(i)).append(' '); - } - if (size > NODE_LIST_LIMIT) { - buffer.append(String.format("...(total %d entries)", size)); - } - buffer.append("]; "); - } - List racks = request.getRacks(); - if (racks != null) { - buffer.append("racks = [") - .append(join(racks, ", ", false)) - .append("]; "); - } - return buffer.toString(); - } - - public static String trimPrefix(String prefix) { - if (prefix != null && prefix.endsWith(COMPONENT_SEPARATOR)) { - return prefix.substring(0, prefix.length()-1); - } - return prefix; - } - public static String createNameTag(String name) { return "Name: " + name; } @@ -2485,64 +1692,8 @@ public static String createDescriptionTag(String description) { return "Description: " + description; } - /** - * Get all YarnApplicationState values which signify that an application is - * in RUNNING or pre-RUNNING state. - * - * @return all live app states - */ - public static EnumSet getAllLiveAppStates() { - return EnumSet.range(YarnApplicationState.NEW, - YarnApplicationState.RUNNING); - } - - /** - * Get all YarnApplicationState values which signify that an application is - * not live, which means it is in one of the post RUNNING states. - * - * @return all non live app states - */ - public static EnumSet getAllNonLiveAppStates() { - return EnumSet.range(YarnApplicationState.FINISHED, - YarnApplicationState.KILLED); - } - public static final String DAYS = ".days"; public static final String HOURS = ".hours"; public static final String MINUTES = ".minutes"; public static final String SECONDS = ".seconds"; - - /** - * Get the time range of a set of keys. - * @param conf configuration to read properties from - * @param basekey base key to which suffix gets applied - * @param defDays - * @param defHours - * @param defMins - * @param defSecs - * @return the aggregate time range in seconds - */ - public static long getTimeRange(org.apache.slider.api.resource - .Configuration conf, - String basekey, - long defDays, - long defHours, - long defMins, - long defSecs) { - Preconditions.checkArgument(basekey != null); - long days = conf.getPropertyLong(basekey + DAYS, defDays); - long hours = conf.getPropertyLong(basekey + HOURS, defHours); - - long minutes = conf.getPropertyLong(basekey + MINUTES, defMins); - long seconds = conf.getPropertyLong(basekey + SECONDS, defSecs); - // range check - Preconditions.checkState(days >= 0 && hours >= 0 && minutes >= 0 - && seconds >= 0, - "Time range for %s has negative time component %s:%s:%s:%s", - basekey, days, hours, minutes, seconds); - - // calculate total time, schedule the reset if expected - long totalMinutes = days * 24 * 60 + hours * 24 + minutes; - return totalMinutes * 60 + seconds; - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZookeeperUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ZookeeperUtils.java similarity index 96% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZookeeperUtils.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ZookeeperUtils.java index cc1b2c9..1fa07ce 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZookeeperUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/yarn/service/utils/ZookeeperUtils.java @@ -16,12 +16,11 @@ * limitations under the License. */ -package org.apache.slider.core.zk; +package org.apache.hadoop.yarn.service.utils; import com.google.common.net.HostAndPort; import org.apache.hadoop.util.StringUtils; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadConfigException; +import org.apache.hadoop.yarn.service.exceptions.BadConfigException; import java.util.ArrayList; import java.util.List; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/Slider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/Slider.java deleted file mode 100644 index 5fc8618..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/Slider.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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.slider; - -import org.apache.slider.client.SliderClient; -import org.apache.slider.core.main.ServiceLauncher; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -/** - * This is just the entry point class - */ -public class Slider extends SliderClient { - - - public static final String SERVICE_CLASSNAME = "org.apache.slider.Slider"; - - /** - * This is the main entry point for the service launcher. - * @param args command line arguments. - */ - public static void main(String[] args) { - - //turn the args to a list - List argsList = Arrays.asList(args); - //create a new list, as the ArrayList type doesn't push() on an insert - List extendedArgs = new ArrayList(argsList); - //insert the service name - extendedArgs.add(0, SERVICE_CLASSNAME); - //now have the service launcher do its work - ServiceLauncher.serviceMain(extendedArgs); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterNode.java deleted file mode 100644 index 8b0a563..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterNode.java +++ /dev/null @@ -1,220 +0,0 @@ -/* - * 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.slider.api; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.slider.api.proto.Messages; -import org.codehaus.jackson.JsonParseException; -import org.codehaus.jackson.annotate.JsonIgnore; -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.JsonMappingException; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.annotate.JsonSerialize; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; - -/** - * Describe a specific node in the cluster - */ -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL ) -public final class ClusterNode implements Cloneable { - protected static final Logger - LOG = LoggerFactory.getLogger(ClusterNode.class); - - @JsonIgnore - public ContainerId containerId; - - /** - * server name - */ - public String name; - - - /** - * UUID of container used in Slider RPC to refer to instances - */ - public String id; - - public String role; - - public int roleId; - - public long createTime; - public long startTime; - /** - * flag set when it is released, to know if it has - * already been targeted for termination - */ - public boolean released; - public String host; - public String ip; - public String hostname; - public String hostUrl; - - /** - * state from {@link ClusterDescription} - */ - public int state; - - /** - * Exit code: only valid if the state >= STOPPED - */ - public int exitCode; - - /** - * what was the command executed? - */ - public String command; - - /** - * Any diagnostics - */ - public String diagnostics; - - /** - * What is the tail output from the executed process (or [] if not started - * or the log cannot be picked up - */ - public String[] output; - - /** - * Any environment details - */ - public String[] environment; - - /** - * server-side ctor takes the container ID and builds the name from it - * @param containerId container ID; can be null - */ - public ClusterNode(ContainerId containerId) { - if (containerId != null) { - this.containerId = containerId; - this.name = containerId.toString(); - } - } - - /** - * ctor for deserialization - */ - public ClusterNode() { - } - - @Override - public String toString() { - StringBuilder builder = new StringBuilder(); - builder.append(name).append(": "); - builder.append(state).append("\n"); - builder.append("state: ").append(state).append("\n"); - builder.append("role: ").append(role).append("\n"); - append(builder, "host", host); - append(builder, "hostURL", hostUrl); - append(builder, "command", command); - if (output != null) { - for (String line : output) { - builder.append(line).append("\n"); - } - } - append(builder, "diagnostics", diagnostics); - return builder.toString(); - } - - private void append(StringBuilder builder, String key, Object val) { - if (val != null) { - builder.append(key).append(": ").append(val.toString()).append("\n"); - } - } - - /** - * Convert to a JSON string - * @return a JSON string description - * @throws IOException Problems mapping/writing the object - */ - public String toJsonString() throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.writeValueAsString(this); - } - - - /** - * Convert from JSON - * @param json input - * @return the parsed JSON - * @throws IOException IO - */ - public static ClusterNode fromJson(String json) - throws IOException, JsonParseException, JsonMappingException { - ObjectMapper mapper = new ObjectMapper(); - try { - return mapper.readValue(json, ClusterNode.class); - } catch (IOException e) { - LOG.error("Exception while parsing json : {}\n{}", e , json, e); - throw e; - } - } - - /** - * Build from a protobuf response - * @param message - * @return the deserialized node - */ - public static ClusterNode fromProtobuf(Messages.RoleInstanceState message) { - ClusterNode node = new ClusterNode(); - node.name = message.getName(); - node.command = message.getCommand(); - node.diagnostics = message.getDiagnostics(); - String[] arr; - int environmentCount = message.getEnvironmentCount(); - if (environmentCount > 0) { - arr = new String[environmentCount]; - node.environment = message.getEnvironmentList().toArray(arr); - } - node.exitCode = message.getExitCode(); - int outputCount = message.getOutputCount(); - if (outputCount > 0) { - arr = new String[outputCount]; - node.output = message.getOutputList().toArray(arr); - } - node.role = message.getRole(); - node.roleId = message.getRoleId(); - node.state = message.getState(); - node.host = message.getHost(); - node.hostUrl = message.getHostURL(); - node.createTime = message.getCreateTime(); - node.startTime = message.getStartTime(); - node.released = message.getReleased(); - return node; - } - - @Override - public Object clone() throws CloneNotSupportedException { - return super.clone(); - } - - public ClusterNode doClone() { - try { - return (ClusterNode)clone(); - } catch (CloneNotSupportedException e) { - //not going to happen. This is a final class - return null; - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/InternalKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/InternalKeys.java deleted file mode 100644 index 0e3b535..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/InternalKeys.java +++ /dev/null @@ -1,210 +0,0 @@ -/* - * 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.slider.api; - -/** - * Keys for internal use, go into `internal.json` and not intended for normal - * use except when tuning Slider AM operations - */ -public interface InternalKeys { - - - /** - * Home dir of the app: {@value} - * If set, implies there is a home dir to use - */ - String INTERNAL_APPLICATION_HOME = "internal.application.home"; - /** - * Path to an image file containing the app: {@value} - */ - String INTERNAL_APPLICATION_IMAGE_PATH = "internal.application.image.path"; - /** - * Time in milliseconds to wait after forking any in-AM - * process before attempting to start up the containers: {@value} - * - * A shorter value brings the cluster up faster, but means that if the - * in AM process fails (due to a bad configuration), then time - * is wasted starting containers on a cluster that isn't going to come - * up - */ - String INTERNAL_CONTAINER_STARTUP_DELAY = "internal.container.startup.delay"; - /** - * internal temp directory: {@value} - */ - String INTERNAL_AM_TMP_DIR = "internal.am.tmp.dir"; - /** - * internal temp directory: {@value} - */ - String INTERNAL_TMP_DIR = "internal.tmp.dir"; - /** - * where a snapshot of the original conf dir is: {@value} - */ - String INTERNAL_SNAPSHOT_CONF_PATH = "internal.snapshot.conf.path"; - /** - * where a snapshot of the original conf dir is: {@value} - */ - String INTERNAL_GENERATED_CONF_PATH = "internal.generated.conf.path"; - /** - * where a snapshot of the original conf dir is: {@value} - */ - String INTERNAL_PROVIDER_NAME = "internal.provider.name"; - /** - * where a snapshot of the original conf dir is: {@value} - */ - String INTERNAL_DATA_DIR_PATH = "internal.data.dir.path"; - /** - * where the app def is stored - */ - String INTERNAL_APPDEF_DIR_PATH = "internal.appdef.dir.path"; - /** - * where addons for the app are stored - */ - String INTERNAL_ADDONS_DIR_PATH = "internal.addons.dir.path"; - /** - * Time in milliseconds to wait after forking any in-AM - * process before attempting to start up the containers: {@value} - * - * A shorter value brings the cluster up faster, but means that if the - * in AM process fails (due to a bad configuration), then time - * is wasted starting containers on a cluster that isn't going to come - * up - */ - int DEFAULT_INTERNAL_CONTAINER_STARTUP_DELAY = 5000; - /** - * Time in seconds before a container is considered long-lived. - * Shortlived containers are interpreted as a problem with the role - * and/or the host: {@value} - */ - String INTERNAL_CONTAINER_FAILURE_SHORTLIFE = - "internal.container.failure.shortlife"; - /** - * Default short life threshold: {@value} - */ - int DEFAULT_INTERNAL_CONTAINER_FAILURE_SHORTLIFE = 60; - - /** - * Version of the app: {@value} - */ - String KEYTAB_LOCATION = "internal.keytab.location"; - - /** - * Queue used to deploy the app: {@value} - */ - String INTERNAL_QUEUE = "internal.queue"; - - /** - * Flag to indicate whether or not the chaos monkey is enabled: - * {@value} - */ - String CHAOS_MONKEY_ENABLED = "internal.chaos.monkey.enabled"; - boolean DEFAULT_CHAOS_MONKEY_ENABLED = false; - - - /** - * Rate - */ - - String CHAOS_MONKEY_INTERVAL = "internal.chaos.monkey.interval"; - String CHAOS_MONKEY_INTERVAL_DAYS = CHAOS_MONKEY_INTERVAL + ".days"; - String CHAOS_MONKEY_INTERVAL_HOURS = CHAOS_MONKEY_INTERVAL + ".hours"; - String CHAOS_MONKEY_INTERVAL_MINUTES = CHAOS_MONKEY_INTERVAL + ".minutes"; - String CHAOS_MONKEY_INTERVAL_SECONDS = CHAOS_MONKEY_INTERVAL + ".seconds"; - - long DEFAULT_CHAOS_MONKEY_INTERVAL_DAYS = 0; - long DEFAULT_CHAOS_MONKEY_INTERVAL_HOURS = 0; - long DEFAULT_CHAOS_MONKEY_INTERVAL_MINUTES = 0; - - String CHAOS_MONKEY_DELAY = "internal.chaos.monkey.delay"; - String CHAOS_MONKEY_DELAY_DAYS = CHAOS_MONKEY_DELAY + ".days"; - String CHAOS_MONKEY_DELAY_HOURS = CHAOS_MONKEY_DELAY + ".hours"; - String CHAOS_MONKEY_DELAY_MINUTES = CHAOS_MONKEY_DELAY + ".minutes"; - String CHAOS_MONKEY_DELAY_SECONDS = CHAOS_MONKEY_DELAY + ".seconds"; - - int DEFAULT_CHAOS_MONKEY_STARTUP_DELAY = 0; - - /** - * Prefix for all chaos monkey probabilities - */ - String CHAOS_MONKEY_PROBABILITY = - "internal.chaos.monkey.probability"; - /** - * Probabilies are out of 10000 ; 100==1% - */ - - /** - * Probability of a monkey check killing the AM: {@value} - */ - String CHAOS_MONKEY_PROBABILITY_AM_FAILURE = - CHAOS_MONKEY_PROBABILITY + ".amfailure"; - - /** - * Default probability of a monkey check killing the AM: {@value} - */ - int DEFAULT_CHAOS_MONKEY_PROBABILITY_AM_FAILURE = 0; - - /** - * Probability of a monkey check killing the AM: {@value} - */ - String CHAOS_MONKEY_PROBABILITY_AM_LAUNCH_FAILURE = - CHAOS_MONKEY_PROBABILITY + ".amlaunchfailure"; - - /** - * Probability of a monkey check killing a container: {@value} - */ - - String CHAOS_MONKEY_PROBABILITY_CONTAINER_FAILURE = - CHAOS_MONKEY_PROBABILITY + ".containerfailure"; - - /** - * Default probability of a monkey check killing the a container: {@value} - */ - int DEFAULT_CHAOS_MONKEY_PROBABILITY_CONTAINER_FAILURE = 0; - - - /** - * 1% of chaos - */ - int PROBABILITY_PERCENT_1 = 100; - - /** - * 100% for chaos values - */ - int PROBABILITY_PERCENT_100 = 100 * PROBABILITY_PERCENT_1; - - /** - * interval between checks for escalation: {@value} - */ - String ESCALATION_CHECK_INTERVAL = "escalation.check.interval.seconds"; - - /** - * default value: {@value} - */ - int DEFAULT_ESCALATION_CHECK_INTERVAL = 30; - - - /** - * interval between readiness checks: {@value} - */ - String MONITOR_INTERVAL = "monitor.interval.seconds"; - - /** - * default value: {@value} - */ - int DEFAULT_MONITOR_INTERVAL = 30; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java deleted file mode 100644 index 988627d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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.slider.api; - -/** - * Keys for entries in the options section - * of a cluster description. - */ -public interface OptionKeys extends InternalKeys { - - /** - * Time in milliseconds to wait after forking any in-AM - * process before attempting to start up the containers: {@value} - * - * A shorter value brings the cluster up faster, but means that if the - * in AM process fails (due to a bad configuration), then time - * is wasted starting containers on a cluster that isn't going to come - * up - */ - String APPLICATION_TYPE = "application.type"; - - String APPLICATION_NAME = "application.name"; - - /** - * Prefix for site.xml options: {@value} - */ - String SITE_XML_PREFIX = "site."; - /** - * Prefix for config file options: {@value} - */ - String CONF_FILE_PREFIX = "conf."; - /** - * Prefix for package options: {@value} - */ - String PKG_FILE_PREFIX = "pkg."; - /** - * Prefix for export options: {@value} - */ - String EXPORT_PREFIX = "export."; - /** - * Type suffix for config file and package options: {@value} - */ - String TYPE_SUFFIX = ".type"; - /** - * Name suffix for config file and package options: {@value} - */ - String NAME_SUFFIX = ".name"; - /** - * Per component suffix for config file options: {@value} - */ - String PER_COMPONENT = ".per.component"; - /** - * Per group suffix for config file options: {@value} - */ - String PER_GROUP = ".per.group"; - - /** - * Zookeeper quorum host list: {@value} - */ - String ZOOKEEPER_QUORUM = "zookeeper.quorum"; - String ZOOKEEPER_HOSTS = "zookeeper.hosts"; - String ZOOKEEPER_PORT = "zookeeper.port"; - - /** - * Zookeeper path value (string): {@value} - */ - String ZOOKEEPER_PATH = "zookeeper.path"; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java deleted file mode 100644 index 2f71004..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java +++ /dev/null @@ -1,210 +0,0 @@ -/* - * 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.slider.api; - -/** - * These are the keys valid in resource options - * - /* - - Container failure window. - - The window is calculated in minutes as as (days * 24 *60 + hours* 24 + minutes) - - Every interval of this period after the AM is started/restarted becomes - the time period in which the CONTAINER_FAILURE_THRESHOLD value is calculated. - - After the window limit is reached, the failure counts are reset. This - is not a sliding window/moving average policy, simply a rule such as - "every six hours the failure count is reset" - - -
- ===========================================================================
- 
- - */ -public interface ResourceKeys { - - - /** - * #of instances of a component: {@value} - * - */ - String COMPONENT_INSTANCES = "yarn.component.instances"; - - /** - * Whether to use unique names for each instance of a component: {@value} - */ - String UNIQUE_NAMES = "component.unique.names"; - - /** - * Amount of memory to ask YARN for in MB. - * Important: this may be a hard limit on the - * amount of RAM that the service can use - * {@value} - */ - String YARN_MEMORY = "yarn.memory"; - - /** {@value} */ - int DEF_YARN_MEMORY = 256; - - /** - * Number of cores/virtual cores to ask YARN for - * {@value} - */ - String YARN_CORES = "yarn.vcores"; - - /** - * If normalization is set to false, then if the resource (memory and/or - * vcore) requested by a role is higher than YARN limits, then the resource - * request is not normalized. If this causes failures at the YARN level then - * applications are expecting that to happen. Default value is true. - */ - String YARN_RESOURCE_NORMALIZATION_ENABLED = - "yarn.resource.normalization.enabled"; - - /** - * Number of disks per instance to ask YARN for - * {@value} - */ - String YARN_DISKS = "yarn.disks.count-per-instance"; - - /** - * Disk size per disk to ask YARN for - * {@value} - */ - String YARN_DISK_SIZE = "yarn.disk.size"; - - /** {@value} */ - int DEF_YARN_CORES = 1; - - - /** - * Label expression that this container must satisfy - * {@value} - */ - String YARN_LABEL_EXPRESSION = "yarn.label.expression"; - - /** default label expression: */ - String DEF_YARN_LABEL_EXPRESSION = null; - - - /** - * Constant to indicate that the requirements of a YARN resource limit - * (cores, memory, ...) should be set to the maximum allowed by - * the queue into which the YARN container requests are placed. - */ - String YARN_RESOURCE_MAX = "max"; - - /** - * Mandatory property for all roles - * 1. this must be defined. - * 2. this must be >= 1 - * 3. this must not match any other role priority in the cluster. - */ - String COMPONENT_PRIORITY = "yarn.role.priority"; - - /** - * placement policy - */ - String COMPONENT_PLACEMENT_POLICY = "yarn.component.placement.policy"; - - /** - * Maximum number of node failures that can be tolerated by a component on a specific node - */ - String NODE_FAILURE_THRESHOLD = - "yarn.node.failure.threshold"; - - /** - * maximum number of failed containers (in a single role) - * before the cluster is deemed to have failed {@value} - */ - String CONTAINER_FAILURE_THRESHOLD = - "yarn.container.failure.threshold"; - - /** - * prefix for the time of the container failure reset window. - * {@value} - */ - - String CONTAINER_FAILURE_WINDOW = - "yarn.container.failure.window"; - - - - long DEFAULT_CONTAINER_FAILURE_WINDOW_DAYS = 0; - long DEFAULT_CONTAINER_FAILURE_WINDOW_HOURS = 6; - long DEFAULT_CONTAINER_FAILURE_WINDOW_MINUTES = 0; - - - /** - * Default failure threshold: {@value} - */ - int DEFAULT_CONTAINER_FAILURE_THRESHOLD = 5; - - /** - * Default node failure threshold for a component instance: {@value} - * Should to be lower than default component failure threshold to allow - * the component to start elsewhere - */ - int DEFAULT_NODE_FAILURE_THRESHOLD = 3; - - /** - * Failure threshold is unlimited: {@value} - */ - int NODE_FAILURE_THRESHOLD_UNLIMITED = -1; - - /** - * Time in seconds to escalate placement delay - */ - String PLACEMENT_ESCALATE_DELAY = - "yarn.placement.escalate.seconds"; - - /** - * Time to have a strict placement policy outstanding before - * downgrading to a lax placement (for those components which permit that). - *
    - *
  1. For strictly placed components, there's no relaxation.
  2. - *
  3. For components with no locality, there's no need to relax
  4. - *
- * - */ - int DEFAULT_PLACEMENT_ESCALATE_DELAY_SECONDS = 30; - - /** - * Log aggregation include, exclude patterns - */ - String YARN_LOG_INCLUDE_PATTERNS = "yarn.log.include.patterns"; - String YARN_LOG_EXCLUDE_PATTERNS = "yarn.log.exclude.patterns"; - - String YARN_PROFILE_NAME = "yarn.resource-profile-name"; - - /** - * Window of time where application master's failure count - * can be reset to 0. - */ - String YARN_RESOURCEMANAGER_AM_RETRY_COUNT_WINDOW_MS = - "yarn.resourcemanager.am.retry-count-window-ms"; - - /** - * The default window for Slider. - */ - long DEFAULT_AM_RETRY_COUNT_WINDOW_MS = 300000; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/RoleKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/RoleKeys.java deleted file mode 100644 index ce413ff..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/RoleKeys.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * 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.slider.api; - -/** - * Standard options for roles - */ -public interface RoleKeys { - - - /** - * The name of a role: {@value} - */ - String ROLE_NAME = "role.name"; - - /** - * The group of a role: {@value} - */ - String ROLE_GROUP = "role.group"; - - /** - * The prefix of a role: {@value} - */ - String ROLE_PREFIX = "role.prefix"; - - /** - * Status report: number actually granted : {@value} - */ - String ROLE_ACTUAL_INSTANCES = "role.actual.instances"; - - /** - * Status report: number currently requested: {@value} - */ - String ROLE_REQUESTED_INSTANCES = "role.requested.instances"; - - /** - * Status report: number currently being released: {@value} - */ - String ROLE_RELEASING_INSTANCES = "role.releasing.instances"; - - /** - * Status report: total number that have failed: {@value} - */ - String ROLE_FAILED_INSTANCES = "role.failed.instances"; - - /** - * Status report: number that have failed recently: {@value} - */ - String ROLE_FAILED_RECENTLY_INSTANCES = "role.failed.recently.instances"; - - /** - * Status report: number that have failed for node-related issues: {@value} - */ - String ROLE_NODE_FAILED_INSTANCES = "role.failed.node.instances"; - - /** - * Status report: number that been pre-empted: {@value} - */ - String ROLE_PREEMPTED_INSTANCES = "role.failed.preempted.instances"; - - /** - * Number of pending anti-affine instances: {@value} - */ - String ROLE_PENDING_AA_INSTANCES = "role.pending.aa.instances"; - - /** - * Status report: number currently being released: {@value} - */ - String ROLE_FAILED_STARTING_INSTANCES = "role.failed.starting.instances"; - - /** - * Extra arguments (non-JVM) to use when starting this role - */ - String ROLE_ADDITIONAL_ARGS = "role.additional.args"; - - /** - * JVM heap size for Java applications in MB. Only relevant for Java applications. - * This MUST be less than or equal to the {@link ResourceKeys#YARN_MEMORY} option - * {@value} - */ - String JVM_HEAP = "jvm.heapsize"; - - /* - * GC options for Java applications. - */ - String GC_OPTS = "gc.opts"; - - /** - * JVM options other than heap size. Only relevant for Java applications. - * {@value} - */ - String JVM_OPTS = "jvm.opts"; - - - /** - * All keys w/ env. are converted into env variables and passed down - */ - String ENV_PREFIX = "env."; - - /** - * Container service record attribute prefix. - */ - String SERVICE_RECORD_ATTRIBUTE_PREFIX = "service.record.attribute"; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/SliderClusterProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/SliderClusterProtocol.java deleted file mode 100644 index aaf2f88..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/SliderClusterProtocol.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * 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") throws IOException, YarnException; 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.slider.api; - -import org.apache.hadoop.ipc.VersionedProtocol; -import org.apache.hadoop.security.KerberosInfo; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.slider.api.proto.Messages; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; - -import java.io.IOException; - -/** - * Cluster protocol. This can currently act as a versioned IPC - * endpoint or be relayed via protobuf - */ -@KerberosInfo(serverPrincipal = SliderXmlConfKeys.KEY_KERBEROS_PRINCIPAL) -public interface SliderClusterProtocol extends VersionedProtocol { - long versionID = 0x01; - - /** - * Stop the cluster - */ - - Messages.StopClusterResponseProto stopCluster(Messages.StopClusterRequestProto request) throws - IOException, YarnException; - /** - * Upgrade the application containers - * - * @param request upgrade containers request object - * @return upgrade containers response object - * @throws IOException - * @throws YarnException - */ - Messages.UpgradeContainersResponseProto upgradeContainers( - Messages.UpgradeContainersRequestProto request) throws IOException, - YarnException; - - - Messages.FlexComponentsResponseProto flexComponents( - Messages.FlexComponentsRequestProto request) throws IOException; - - /** - * Get the current cluster status - */ - Messages.GetJSONClusterStatusResponseProto getJSONClusterStatus(Messages.GetJSONClusterStatusRequestProto request) - throws IOException, YarnException; - - - /** - * List all running nodes in a role - */ - Messages.ListNodeUUIDsByRoleResponseProto listNodeUUIDsByRole(Messages.ListNodeUUIDsByRoleRequestProto request) - throws IOException, YarnException; - - - /** - * Get the details on a node - */ - Messages.GetNodeResponseProto getNode(Messages.GetNodeRequestProto request) - throws IOException, YarnException; - - /** - * Get the - * details on a list of nodes. - * Unknown nodes are not returned - * Important: the order of the results are undefined - */ - Messages.GetClusterNodesResponseProto getClusterNodes(Messages.GetClusterNodesRequestProto request) - throws IOException, YarnException; - - /** - * Echo back the submitted text (after logging it). - * Useful for adding information to the log, and for testing round trip - * operations of the protocol - * @param request request - * @return response - * @throws IOException - * @throws YarnException - */ - Messages.EchoResponseProto echo(Messages.EchoRequestProto request) throws IOException, YarnException; - - /** - * Kill an identified container - * @param request request containing the container to kill - * @return the response - * @throws IOException - * @throws YarnException - */ - Messages.KillContainerResponseProto killContainer(Messages.KillContainerRequestProto request) - throws IOException, YarnException; - - /** - * AM to commit suicide. If the Hadoop halt entry point has not been disabled, - * this will fail rather than return with a response. - * @param request request - * @return response (this is not the expected outcome) - * @throws IOException - * @throws YarnException - */ - Messages.AMSuicideResponseProto amSuicide(Messages.AMSuicideRequestProto request) - throws IOException; - - /** - * Get the application liveness - * @return current liveness information - * @throws IOException - */ - Messages.ApplicationLivenessInformationProto getLivenessInformation( - Messages.GetApplicationLivenessRequestProto request - ) throws IOException; - - Messages.GetLiveContainersResponseProto getLiveContainers( - Messages.GetLiveContainersRequestProto request - ) throws IOException; - - Messages.ContainerInformationProto getLiveContainer( - Messages.GetLiveContainerRequestProto request - ) throws IOException; - - Messages.GetLiveComponentsResponseProto getLiveComponents( - Messages.GetLiveComponentsRequestProto request - ) throws IOException; - - Messages.ComponentInformationProto getLiveComponent( - Messages.GetLiveComponentRequestProto request - ) throws IOException; - - Messages.GetLiveNodesResponseProto getLiveNodes( - Messages.GetLiveNodesRequestProto request - ) throws IOException; - - Messages.NodeInformationProto getLiveNode( - Messages.GetLiveNodeRequestProto request - ) throws IOException; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StateValues.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StateValues.java deleted file mode 100644 index ad66a97..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StateValues.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.slider.api; - -/** - * Enumeration of state values. - */ -public class StateValues { - - private StateValues() {} - - /** - * Specification is incomplete & cannot - * be used: {@value}. - */ - public static final int STATE_INCOMPLETE = 0; - - /** - * Spec has been submitted: {@value} - */ - public static final int STATE_SUBMITTED = 1; - /** - * Cluster created: {@value} - */ - public static final int STATE_CREATED = 2; - /** - * Live: {@value} - */ - public static final int STATE_LIVE = 3; - /** - * Not ready. - */ - public static final int STATE_NOT_READY = 4; - /** - * Ready. - */ - public static final int STATE_READY = 5; - /** - * Stopped. - */ - public static final int STATE_STOPPED = 99; - /** - * Destroyed. - */ - public static final int STATE_DESTROYED = 100; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StatusKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StatusKeys.java deleted file mode 100644 index 8a2c4bb..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StatusKeys.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * 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.slider.api; -import static org.apache.slider.api.ResourceKeys.COMPONENT_INSTANCES; -/** - * Contains status and statistics keys - */ -public interface StatusKeys { - - String STATISTICS_CONTAINERS_ACTIVE_REQUESTS = "containers.active.requests"; - String STATISTICS_CONTAINERS_COMPLETED = "containers.completed"; - String STATISTICS_CONTAINERS_DESIRED = "containers.desired"; - String STATISTICS_CONTAINERS_FAILED = "containers.failed"; - String STATISTICS_CONTAINERS_FAILED_RECENTLY = "containers.failed.recently"; - String STATISTICS_CONTAINERS_FAILED_NODE = "containers.failed.node"; - String STATISTICS_CONTAINERS_PREEMPTED = "containers.failed.preempted"; - String STATISTICS_CONTAINERS_LIVE = "containers.live"; - String STATISTICS_CONTAINERS_REQUESTED = "containers.requested"; - String STATISTICS_CONTAINERS_ANTI_AFFINE_PENDING = "containers.anti-affine.pending"; - String STATISTICS_CONTAINERS_STARTED = "containers.start.started"; - String STATISTICS_CONTAINERS_START_FAILED = - "containers.start.failed"; - String STATISTICS_CONTAINERS_SURPLUS = - "containers.surplus"; - String STATISTICS_CONTAINERS_UNKNOWN_COMPLETED = - "containers.unknown.completed"; - /** - * No of containers provided on AM restart - */ - String INFO_CONTAINERS_AM_RESTART = "containers.at.am-restart"; - - String INFO_CREATE_TIME_MILLIS = "create.time.millis"; - String INFO_CREATE_TIME_HUMAN = "create.time"; - String INFO_LIVE_TIME_MILLIS = "live.time.millis"; - String INFO_LIVE_TIME_HUMAN = "live.time"; - String INFO_FLEX_TIME_MILLIS = "flex.time.millis"; - String INFO_FLEX_TIME_HUMAN = "flex.time"; - - String INFO_MASTER_ADDRESS = "info.master.address"; - - /** - * System time in millis when the status report was generated - */ - String INFO_STATUS_TIME_MILLIS = "status.time.millis"; - - /** - * System time in human form when the status report was generated - */ - String INFO_STATUS_TIME_HUMAN = "status.time"; - - String INFO_AM_APP_ID = "info.am.app.id"; - String INFO_AM_ATTEMPT_ID = "info.am.attempt.id"; - String INFO_AM_CONTAINER_ID = "info.am.container.id"; - String INFO_AM_HOSTNAME = "info.am.hostname"; - String INFO_AM_RPC_PORT = "info.am.rpc.port"; - String INFO_AM_WEB_PORT = "info.am.web.port"; - String INFO_AM_WEB_URL = "info.am.web.url"; - String INFO_AM_AGENT_STATUS_PORT = "info.am.agent.status.port"; - String INFO_AM_AGENT_OPS_PORT = "info.am.agent.ops.port"; - String INFO_AM_AGENT_OPS_URL = "info.am.agent.ops.url"; - String INFO_AM_AGENT_STATUS_URL = "info.am.agent.status.url"; - - /** - * info: #of instances of a component requested: {@value} - * - */ - String COMPONENT_INSTANCES_ACTUAL = COMPONENT_INSTANCES + ".actual"; - - /** - * info: #of instances of a component requested: {@value} - * - */ - String COMPONENT_INSTANCES_REQUESTING = COMPONENT_INSTANCES + ".requesting"; - - /** - * info: #of instances of a component being released: {@value} - * - */ - String COMPONENT_INSTANCES_RELEASING = COMPONENT_INSTANCES + ".releasing"; - - /** - * info: #of instances of a component failed: {@value} - * - */ - String COMPONENT_INSTANCES_FAILED = COMPONENT_INSTANCES + ".failed"; - - /** - * info: #of instances of a component started: {@value} - * - */ - String COMPONENT_INSTANCES_STARTED = COMPONENT_INSTANCES + ".started"; - - - /** - * info: #of instances of a component completed: {@value} - * - */ - String COMPONENT_INSTANCES_COMPLETED = COMPONENT_INSTANCES + ".completed"; - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ApplicationLivenessInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ApplicationLivenessInformation.java deleted file mode 100644 index 687edd2..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ApplicationLivenessInformation.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.slider.api.types; - -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.annotate.JsonSerialize; - -/** - * Serialized information about liveness - *

- * If true liveness probes are implemented, this - * datatype can be extended to publish their details. - */ -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL) -public class ApplicationLivenessInformation { - /** flag set if the cluster is at size */ - public boolean allRequestsSatisfied; - - /** number of outstanding requests: those needed to satisfy */ - public int requestsOutstanding; - - @Override - public String toString() { - final StringBuilder sb = - new StringBuilder("ApplicationLivenessInformation{"); - sb.append("allRequestsSatisfied=").append(allRequestsSatisfied); - sb.append(", requestsOutstanding=").append(requestsOutstanding); - sb.append('}'); - return sb.toString(); - } -} - - diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ComponentInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ComponentInformation.java deleted file mode 100644 index d2fdd62..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ComponentInformation.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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.slider.api.types; - -import org.apache.slider.api.StatusKeys; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.annotate.JsonSerialize; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * Serializable version of component data. - *

- * This is sent in REST calls as a JSON object —but is also marshalled into - * a protobuf structure. Look at {@link RestTypeMarshalling} - * for the specifics there. - *

- * This means that if any fields are added here. they must be added to - * src/main/proto/SliderClusterMessages.proto and - * the protobuf structures rebuilt via a {@code mvn generate-sources -Pcompile-protobuf} - * - * See also {@link RoleStatus#serialize()} - */ -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL) - -public class ComponentInformation { - - public String name; - public int priority; - public int desired, actual, releasing; - public int placementPolicy; - public int requested; - public int failed, started, startFailed, completed, totalRequested; - public int nodeFailed, failedRecently, preempted; - public int pendingAntiAffineRequestCount; - public boolean isAARequestOutstanding; - - public String failureMessage; - public List containers; - - /** - * Build the statistics map from the current data - * @return a map for use in statistics reports - */ - public Map buildStatistics() { - Map stats = new HashMap<>(); - stats.put(StatusKeys.STATISTICS_CONTAINERS_ACTIVE_REQUESTS, requested); - stats.put(StatusKeys.STATISTICS_CONTAINERS_ANTI_AFFINE_PENDING, pendingAntiAffineRequestCount); - stats.put(StatusKeys.STATISTICS_CONTAINERS_COMPLETED, completed); - stats.put(StatusKeys.STATISTICS_CONTAINERS_DESIRED, desired); - stats.put(StatusKeys.STATISTICS_CONTAINERS_FAILED, failed); - stats.put(StatusKeys.STATISTICS_CONTAINERS_FAILED_NODE, nodeFailed); - stats.put(StatusKeys.STATISTICS_CONTAINERS_FAILED_RECENTLY, failedRecently); - stats.put(StatusKeys.STATISTICS_CONTAINERS_LIVE, actual); - stats.put(StatusKeys.STATISTICS_CONTAINERS_PREEMPTED, preempted); - stats.put(StatusKeys.STATISTICS_CONTAINERS_REQUESTED, totalRequested); - stats.put(StatusKeys.STATISTICS_CONTAINERS_STARTED, started); - stats.put(StatusKeys.STATISTICS_CONTAINERS_START_FAILED, startFailed); - return stats; - } - - @Override - public String toString() { - final StringBuilder sb = - new StringBuilder("ComponentInformation{"); - sb.append(", name='").append(name).append('\''); - sb.append(", actual=").append(actual); - sb.append(", completed=").append(completed); - sb.append(", desired=").append(desired); - sb.append(", failed=").append(failed); - sb.append(", failureMessage='").append(failureMessage).append('\''); - sb.append(", placementPolicy=").append(placementPolicy); - sb.append(", isAARequestOutstanding=").append(isAARequestOutstanding); - sb.append(", pendingAntiAffineRequestCount=").append(pendingAntiAffineRequestCount); - sb.append(", priority=").append(priority); - sb.append(", releasing=").append(releasing); - sb.append(", requested=").append(requested); - sb.append(", started=").append(started); - sb.append(", startFailed=").append(startFailed); - sb.append(", totalRequested=").append(totalRequested); - sb.append(", container count='") - .append(containers == null ? 0 : containers.size()) - .append('\''); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ContainerInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ContainerInformation.java deleted file mode 100644 index 6991340..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ContainerInformation.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.slider.api.types; - -import org.apache.hadoop.registry.client.binding.JsonSerDeser; -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.annotate.JsonSerialize; - -/** - * Serializable version of component instance data - */ -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL) -public class ContainerInformation { - - public String containerId; - public String component; - public String appVersion; - public Boolean released; - public int state; - public Integer exitCode; - public String diagnostics; - public long createTime; - public long startTime; - - public String host; - public String hostURL; - public String placement; - /** - * What is the tail output from the executed process (or [] if not started - * or the log cannot be picked up - */ - public String[] output; - - @Override - public String toString() { - JsonSerDeser serDeser = - new JsonSerDeser<>( - ContainerInformation.class); - return serDeser.toString(this); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeEntryInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeEntryInformation.java deleted file mode 100644 index 8424be2..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeEntryInformation.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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.slider.api.types; - -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.annotate.JsonSerialize; - -/** - * Serialized node entry information. Must be kept in sync with the protobuf equivalent. - */ -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL) -public class NodeEntryInformation { - - /** incrementing counter of instances that failed */ - public int failed; - - /** Counter of "failed recently" events. */ - public int failedRecently; - - /** timestamp of last use */ - public long lastUsed; - - /** Number of live nodes. */ - public int live; - - /** incrementing counter of instances that have been pre-empted. */ - public int preempted; - - /** Priority */ - public int priority; - - /** instance explicitly requested on this node */ - public int requested; - - /** number of containers being released off this node */ - public int releasing; - - /** incrementing counter of instances that failed to start */ - public int startFailed; - - /** number of starting instances */ - public int starting; - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder( - "NodeEntryInformation{"); - sb.append("priority=").append(priority); - sb.append(", live=").append(live); - sb.append(", requested=").append(requested); - sb.append(", releasing=").append(releasing); - sb.append(", starting=").append(starting); - sb.append(", failed=").append(failed); - sb.append(", failedRecently=").append(failedRecently); - sb.append(", startFailed=").append(startFailed); - sb.append(", preempted=").append(preempted); - sb.append(", lastUsed=").append(lastUsed); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformation.java deleted file mode 100644 index 4fe5b4c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformation.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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.slider.api.types; - -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.annotate.JsonSerialize; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * Serialized node information. Must be kept in sync with the protobuf equivalent. - */ -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL) -public class NodeInformation { - - public String hostname; - public String state; - public String labels; - public String rackName; - public String httpAddress; - public String healthReport; - public long lastUpdated; - public Map entries = new HashMap<>(); - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder( - "NodeInformation{"); - sb.append("hostname='").append(hostname).append('\''); - sb.append(", state='").append(state).append('\''); - sb.append(", labels='").append(labels).append('\''); - sb.append(", rackName='").append(rackName).append('\''); - sb.append(", httpAddress='").append(httpAddress).append('\''); - sb.append(", healthReport='").append(healthReport).append('\''); - sb.append(", lastUpdated=").append(lastUpdated); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformationList.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformationList.java deleted file mode 100644 index 741523e..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformationList.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.slider.api.types; - -import org.apache.slider.core.persist.JsonSerDeser; - -import java.util.ArrayList; -import java.util.Collection; - -public class NodeInformationList extends ArrayList { - public NodeInformationList() { - } - - public NodeInformationList(Collection c) { - super(c); - } - - public NodeInformationList(int initialCapacity) { - super(initialCapacity); - } - - public static JsonSerDeser createSerializer() { - return new JsonSerDeser<>(NodeInformationList.class); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/PingInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/PingInformation.java deleted file mode 100644 index 223edca..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/PingInformation.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.slider.api.types; - -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.annotate.JsonSerialize; - -/** - * Serialized information to/from Ping operations - */ -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL) -public class PingInformation { - public long time; - public String text; - public String verb; - public String body; - - @Override - public String toString() { - - final StringBuilder sb = - new StringBuilder("PingResource{"); - sb.append("time=").append(time); - sb.append(", verb=").append(verb); - sb.append(", text='").append(text).append('\''); - sb.append(", body='").append(body).append('\''); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/RestTypeMarshalling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/RestTypeMarshalling.java deleted file mode 100644 index bc3d526..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/RestTypeMarshalling.java +++ /dev/null @@ -1,257 +0,0 @@ -/* - * 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.slider.api.types; - -import org.apache.slider.api.proto.Messages; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * Class to handle marshalling of REST - * types to/from Protobuf records. - */ -public class RestTypeMarshalling { - - public static Messages.ApplicationLivenessInformationProto - marshall(ApplicationLivenessInformation info) { - - Messages.ApplicationLivenessInformationProto.Builder builder = - Messages.ApplicationLivenessInformationProto.newBuilder(); - builder.setAllRequestsSatisfied(info.allRequestsSatisfied); - builder.setRequestsOutstanding(info.requestsOutstanding); - return builder.build(); - } - - public static ApplicationLivenessInformation - unmarshall(Messages.ApplicationLivenessInformationProto wire) { - ApplicationLivenessInformation info = new ApplicationLivenessInformation(); - info.allRequestsSatisfied = wire.getAllRequestsSatisfied(); - info.requestsOutstanding = wire.getRequestsOutstanding(); - return info; - } - - public static ComponentInformation - unmarshall(Messages.ComponentInformationProto wire) { - ComponentInformation info = new ComponentInformation(); - info.name = wire.getName(); - info.priority = wire.getPriority(); - info.placementPolicy = wire.getPlacementPolicy(); - - info.actual = wire.getActual(); - info.completed = wire.getCompleted(); - info.desired = wire.getDesired(); - info.failed = wire.getFailed(); - info.releasing = wire.getReleasing(); - info.requested = wire.getRequested(); - info.started = wire.getStarted(); - info.startFailed = wire.getStartFailed(); - info.totalRequested = wire.getTotalRequested(); - info.containers = new ArrayList<>(wire.getContainersList()); - if (wire.hasFailureMessage()) { - info.failureMessage = wire.getFailureMessage(); - } - if (wire.hasPendingAntiAffineRequestCount()) { - info.pendingAntiAffineRequestCount = wire.getPendingAntiAffineRequestCount(); - } - if (wire.hasIsAARequestOutstanding()) { - info.isAARequestOutstanding = wire.getIsAARequestOutstanding(); - } - return info; - } - public static Messages.ComponentInformationProto marshall(ComponentInformation info) { - - Messages.ComponentInformationProto.Builder builder = - Messages.ComponentInformationProto.newBuilder(); - builder.setName(info.name); - builder.setPriority(info.priority); - builder.setPlacementPolicy(info.placementPolicy); - - builder.setActual(info.actual); - builder.setCompleted(info.completed); - builder.setDesired(info.desired); - builder.setFailed(info.failed); - builder.setReleasing(info.releasing); - builder.setRequested(info.requested); - builder.setStarted(info.started); - builder.setStartFailed(info.startFailed); - builder.setTotalRequested(info.totalRequested); - builder.setNodeFailed(info.nodeFailed); - builder.setPreempted(info.preempted); - builder.setFailedRecently(info.failedRecently); - if (info.failureMessage != null) { - builder.setFailureMessage(info.failureMessage); - } - if (info.containers != null) { - builder.addAllContainers(info.containers); - } - builder.setPendingAntiAffineRequestCount(info.pendingAntiAffineRequestCount); - builder.setIsAARequestOutstanding(info.isAARequestOutstanding); - return builder.build(); - } - - public static Messages.NodeInformationProto marshall(NodeInformation info) { - - Messages.NodeInformationProto.Builder builder = - Messages.NodeInformationProto.newBuilder(); - builder.setHostname(info.hostname); - builder.setLastUpdated(info.lastUpdated); - builder.setState(info.state != null? info.state : "unknown"); - builder.setRackName(info.rackName != null ? info.rackName : ""); - builder.setHealthReport(info.healthReport != null ? info.healthReport : ""); - builder.setHttpAddress(info.httpAddress != null ? info.httpAddress : ""); - builder.setLabels(info.labels != null ? info.labels: ""); - - - if (info.entries != null) { - for (Map.Entry elt : info.entries.entrySet()) { - NodeEntryInformation entry = elt.getValue(); - Messages.NodeEntryInformationProto.Builder node = - Messages.NodeEntryInformationProto.newBuilder(); - node.setPriority(entry.priority); - node.setName(elt.getKey()); - node.setFailed(entry.failed); - node.setFailedRecently(entry.failedRecently); - node.setLive(entry.live); - node.setLastUsed(entry.lastUsed); - node.setPreempted(entry.preempted); - node.setRequested(entry.requested); - node.setReleasing(entry.releasing); - node.setStartFailed(entry.startFailed); - node.setStarting(entry.starting); - builder.addEntries(node.build()); - } - } - return builder.build(); - } - - public static NodeInformation unmarshall(Messages.NodeInformationProto wire) { - NodeInformation info = new NodeInformation(); - info.healthReport = wire.getHealthReport(); - info.hostname = wire.getHostname(); - info.httpAddress = wire.getHttpAddress(); - info.labels = wire.getLabels(); - info.lastUpdated = wire.getLastUpdated(); - info.rackName = wire.getRackName(); - info.state = wire.getState(); - List entriesList = wire.getEntriesList(); - if (entriesList != null) { - info.entries = new HashMap<>(entriesList.size()); - for (Messages.NodeEntryInformationProto entry : entriesList) { - NodeEntryInformation nei = new NodeEntryInformation(); - nei.failed = entry.getFailed(); - nei.failedRecently = entry.getFailedRecently(); - nei.lastUsed = entry.getLastUsed(); - nei.live = entry.getLive(); - nei.preempted = entry.getPreempted(); - nei.priority = entry.getPriority(); - nei.requested = entry.getRequested(); - nei.releasing = entry.getReleasing(); - nei.startFailed = entry.getStartFailed(); - nei.starting = entry.getStarting(); - info.entries.put(entry.getName(), nei); - } - } - return info; - } - - public static ContainerInformation unmarshall(Messages.ContainerInformationProto wire) { - ContainerInformation info = new ContainerInformation(); - info.containerId = wire.getContainerId(); - info.component = wire.getComponent(); - info.appVersion = wire.getAppVersion(); - info.state = wire.getState(); - if (wire.hasReleased()) { - info.released = wire.getReleased(); - } - if (wire.hasExitCode()) { - info.exitCode = wire.getExitCode(); - } - if (wire.hasDiagnostics()) { - info.diagnostics = wire.getDiagnostics(); - } - if (wire.hasHost()) { - info.host = wire.getHost(); - } - if (wire.hasHostURL()) { - info.host = wire.getHostURL(); - } - info.createTime = wire.getCreateTime(); - info.startTime = wire.getStartTime(); - info.output = wire.getOutputList().toArray( - new String[wire.getOutputCount()] - ); - if (wire.hasPlacement()) { - info.placement = wire.getPlacement(); - } - return info; - } - - public static List unmarshall(Messages.GetLiveContainersResponseProto wire) { - List infoList = new ArrayList<>(wire.getContainersList().size()); - for (Messages.ContainerInformationProto container : wire.getContainersList()) { - infoList.add(unmarshall(container)); - } - return infoList; - } - - public static Messages.ContainerInformationProto marshall(ContainerInformation info) { - - Messages.ContainerInformationProto.Builder builder = - Messages.ContainerInformationProto.newBuilder(); - if (info.containerId != null) { - builder.setContainerId(info.containerId); - } - if (info.component != null) { - builder.setComponent(info.component); - } - if (info.appVersion != null) { - builder.setAppVersion(info.appVersion); - } - builder.setCreateTime(info.createTime); - if (info.diagnostics != null) { - builder.setDiagnostics(info.diagnostics); - } - if (info.host != null) { - builder.setHost(info.host); - } - if (info.hostURL != null) { - builder.setHostURL(info.hostURL); - } - if (info.output != null) { - builder.addAllOutput(Arrays.asList(info.output)); - } - if (info.released != null) { - builder.setReleased(info.released); - } - if (info.placement != null) { - builder.setPlacement(info.placement); - } - builder.setStartTime(info.startTime); - builder.setState(info.state); - return builder.build(); - } - - public static String unmarshall(Messages.WrappedJsonProto wire) { - return wire.getJson(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/RoleStatistics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/RoleStatistics.java deleted file mode 100644 index 25f4d9d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/RoleStatistics.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.slider.api.types; - -import org.codehaus.jackson.annotate.JsonIgnoreProperties; - -/** - * Simple role statistics for state views; can be generated by RoleStatus - * instances, and aggregated for summary information. - */ -@JsonIgnoreProperties(ignoreUnknown = true) -public class RoleStatistics { - public long activeAA = 0L; - public long actual = 0L; - public long completed = 0L; - public long desired = 0L; - public long failed = 0L; - public long failedRecently = 0L; - public long limitsExceeded = 0L; - public long nodeFailed = 0L; - public long preempted = 0L; - public long requested = 0L; - public long started = 0L; - - /** - * Add another statistics instance - * @param that the other value - * @return this entry - */ - public RoleStatistics add(final RoleStatistics that) { - activeAA += that.activeAA; - actual += that.actual; - completed += that.completed; - desired += that.desired; - failed += that.failed; - failedRecently += that.failedRecently; - limitsExceeded += that.limitsExceeded; - nodeFailed += that.nodeFailed; - preempted += that.preempted; - requested += that.requested; - started += that.started; - return this; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/SliderInstanceDescription.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/SliderInstanceDescription.java deleted file mode 100644 index 3b95f80..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/SliderInstanceDescription.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.slider.api.types; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.yarn.api.records.ApplicationReport; - -/** - * Description of a slider instance - */ -public class SliderInstanceDescription { - - public final String name; - public final Path path; - public final ApplicationReport applicationReport; - - public SliderInstanceDescription(String name, - Path path, - ApplicationReport applicationReport) { - this.name = name; - this.path = path; - this.applicationReport = applicationReport; - } - - @Override - public String toString() { - final StringBuilder sb = - new StringBuilder("SliderInstanceDescription{"); - sb.append("name='").append(name).append('\''); - sb.append(", path=").append(path); - sb.append(", applicationReport: ") - .append(applicationReport == null - ? "null" - : (" id " + applicationReport.getApplicationId())); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientUtils.java deleted file mode 100644 index b28257f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientUtils.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * 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.slider.client; - -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.PathNotFoundException; -import org.apache.hadoop.registry.client.api.RegistryOperations; -import org.apache.hadoop.registry.client.binding.RegistryPathUtils; -import org.apache.hadoop.registry.client.exceptions.NoRecordException; -import org.apache.hadoop.registry.client.types.ServiceRecord; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.NotFoundException; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.registry.docstore.ConfigFormat; -import org.apache.slider.core.registry.docstore.PublishedConfigSet; -import org.apache.slider.core.registry.docstore.PublishedConfiguration; -import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter; -import org.apache.slider.core.registry.retrieve.RegistryRetriever; - -import java.io.File; -import java.io.IOException; - -import static org.apache.hadoop.registry.client.binding.RegistryUtils.currentUser; -import static org.apache.hadoop.registry.client.binding.RegistryUtils.servicePath; - -public class ClientUtils { - public static ServiceRecord lookupServiceRecord(RegistryOperations rops, - String user, String name) throws IOException, SliderException { - return lookupServiceRecord(rops, user, null, name); - } - - public static ServiceRecord lookupServiceRecord(RegistryOperations rops, - String user, String type, String name) throws IOException, - SliderException { - if (StringUtils.isEmpty(user)) { - user = currentUser(); - } else { - user = RegistryPathUtils.encodeForRegistry(user); - } - if (StringUtils.isEmpty(type)) { - type = SliderKeys.APP_TYPE; - } - - String path = servicePath(user, type, name); - return resolve(rops, path); - } - - public static ServiceRecord resolve(RegistryOperations rops, String path) - throws IOException, SliderException { - try { - return rops.resolve(path); - } catch (PathNotFoundException | NoRecordException e) { - throw new NotFoundException(e.getPath().toString(), e); - } - } - - public static PublishedConfiguration getConfigFromRegistry( - RegistryOperations rops, Configuration configuration, - String configName, String appName, String user, boolean external) - throws IOException, SliderException { - ServiceRecord instance = lookupServiceRecord(rops, user, appName); - - RegistryRetriever retriever = new RegistryRetriever(configuration, instance); - PublishedConfigSet configurations = retriever.getConfigurations(external); - - PublishedConfiguration published = retriever.retrieveConfiguration( - configurations, configName, external); - return published; - } - - public static String saveOrReturnConfig(PublishedConfiguration published, - String format, File destPath, String fileName) - throws BadCommandArgumentsException, IOException { - ConfigFormat configFormat = ConfigFormat.resolve(format); - if (configFormat == null) { - throw new BadCommandArgumentsException( - "Unknown/Unsupported format %s ", format); - } - PublishedConfigurationOutputter outputter = - PublishedConfigurationOutputter.createOutputter(configFormat, - published); - boolean print = destPath == null; - if (!print) { - if (destPath.isDirectory()) { - // creating it under a directory - destPath = new File(destPath, fileName); - } - outputter.save(destPath); - return null; - } else { - return outputter.asString(); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java deleted file mode 100644 index 7712191..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java +++ /dev/null @@ -1,2783 +0,0 @@ -/* - * 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.slider.client; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.commons.lang.ArrayUtils; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathNotFoundException; -import org.apache.hadoop.fs.RemoteIterator; -import org.apache.hadoop.fs.permission.FsAction; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto; -import org.apache.hadoop.hdfs.HdfsConfiguration; -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.binding.RegistryPathUtils; -import org.apache.hadoop.registry.client.binding.RegistryUtils; -import org.apache.hadoop.registry.client.exceptions.NoRecordException; -import org.apache.hadoop.registry.client.types.Endpoint; -import org.apache.hadoop.registry.client.types.RegistryPathStatus; -import org.apache.hadoop.registry.client.types.ServiceRecord; -import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes; -import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.security.KerberosDiags; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.util.Shell; -import org.apache.hadoop.yarn.api.ApplicationConstants; -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.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.YarnClientApplication; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException; -import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.ipc.YarnRPC; -import org.apache.hadoop.yarn.util.ConverterUtils; -import org.apache.hadoop.yarn.util.Records; -import org.apache.hadoop.yarn.util.Times; -import org.apache.hadoop.yarn.util.resource.ResourceCalculator; -import org.apache.slider.api.SliderClusterProtocol; -import org.apache.slider.api.proto.Messages; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.api.types.ContainerInformation; -import org.apache.slider.api.types.NodeInformationList; -import org.apache.slider.client.ipc.SliderClusterOperations; -import org.apache.slider.common.Constants; -import org.apache.hadoop.yarn.service.conf.SliderExitCodes; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; -import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; -import org.apache.slider.common.params.AbstractClusterBuildingActionArgs; -import org.apache.slider.common.params.ActionAMSuicideArgs; -import org.apache.slider.common.params.ActionClientArgs; -import org.apache.hadoop.yarn.service.client.params.ActionDependencyArgs; -import org.apache.slider.common.params.ActionDiagnosticArgs; -import org.apache.slider.common.params.ActionExistsArgs; -import org.apache.hadoop.yarn.service.client.params.ActionFlexArgs; -import org.apache.slider.common.params.ActionFreezeArgs; -import org.apache.slider.common.params.ActionKDiagArgs; -import org.apache.slider.common.params.ActionKeytabArgs; -import org.apache.slider.common.params.ActionKillContainerArgs; -import org.apache.slider.common.params.ActionListArgs; -import org.apache.slider.common.params.ActionLookupArgs; -import org.apache.slider.common.params.ActionNodesArgs; -import org.apache.slider.common.params.ActionRegistryArgs; -import org.apache.slider.common.params.ActionResolveArgs; -import org.apache.slider.common.params.ActionResourceArgs; -import org.apache.slider.common.params.ActionStatusArgs; -import org.apache.slider.common.params.ActionThawArgs; -import org.apache.slider.common.params.ActionTokensArgs; -import org.apache.slider.common.params.ActionUpgradeArgs; -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.slider.common.tools.ConfigHelper; -import org.apache.slider.common.tools.Duration; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.common.tools.SliderVersionInfo; -import org.apache.slider.core.exceptions.BadClusterStateException; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.core.exceptions.ErrorStrings; -import org.apache.slider.core.exceptions.NoSuchNodeException; -import org.apache.slider.core.exceptions.NotFoundException; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.exceptions.UnknownApplicationInstanceException; -import org.apache.slider.core.exceptions.UsageException; -import org.apache.slider.core.launch.ClasspathConstructor; -import org.apache.slider.core.launch.CredentialUtils; -import org.apache.slider.core.launch.JavaCommandLineBuilder; -import org.apache.slider.core.launch.SerializedApplicationReport; -import org.apache.slider.core.main.RunService; -import org.apache.slider.core.persist.ApplicationReportSerDeser; -import org.apache.slider.core.persist.JsonSerDeser; -import org.apache.slider.core.registry.SliderRegistryUtils; -import org.apache.slider.core.registry.YarnAppListClient; -import org.apache.slider.core.registry.docstore.ConfigFormat; -import org.apache.slider.core.registry.docstore.PublishedConfigSet; -import org.apache.slider.core.registry.docstore.PublishedConfiguration; -import org.apache.slider.core.registry.docstore.PublishedExports; -import org.apache.slider.core.registry.docstore.PublishedExportsOutputter; -import org.apache.slider.core.registry.docstore.PublishedExportsSet; -import org.apache.slider.core.registry.retrieve.RegistryRetriever; -import org.apache.slider.core.zk.BlockingZKWatcher; -import org.apache.slider.core.zk.ZKIntegration; -import org.apache.hadoop.yarn.service.provider.AbstractClientProvider; -import org.apache.hadoop.yarn.service.provider.ProviderUtils; -import org.apache.slider.server.appmaster.rpc.RpcBinder; -import org.apache.hadoop.yarn.service.ClientAMProtocol; -import org.apache.hadoop.yarn.service.client.ClientAMProxy; -import org.apache.hadoop.yarn.service.ServiceMaster; -import org.apache.slider.server.services.utility.AbstractSliderLaunchedService; -import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooDefs; -import org.apache.zookeeper.data.ACL; -import org.codehaus.jackson.map.PropertyNamingStrategy; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InterruptedIOException; -import java.io.OutputStreamWriter; -import java.io.PrintStream; -import java.io.PrintWriter; -import java.net.InetSocketAddress; -import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import static org.apache.hadoop.registry.client.binding.RegistryUtils.*; -import static org.apache.hadoop.yarn.api.records.YarnApplicationState.*; -import static org.apache.slider.common.Constants.HADOOP_JAAS_DEBUG; -import static org.apache.hadoop.yarn.service.client.params.SliderActions.*; -import static org.apache.slider.common.tools.SliderUtils.*; -import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto; - -/** - * Client service for Slider - */ - -public class SliderClient extends AbstractSliderLaunchedService implements RunService, - SliderExitCodes, SliderKeys, ErrorStrings, SliderClientAPI { - private static final Logger log = LoggerFactory.getLogger(SliderClient.class); - public static final String E_MUST_BE_A_VALID_JSON_FILE - = "Invalid configuration. Must be a valid json file."; - public static final String E_INVALID_INSTALL_LOCATION - = "A valid install location must be provided for the client."; - public static final String E_UNABLE_TO_READ_SUPPLIED_PACKAGE_FILE - = "Unable to read supplied package file"; - public static final String E_INVALID_APPLICATION_PACKAGE_LOCATION - = "A valid application package location required."; - public static final String E_INVALID_INSTALL_PATH = "Install path is not a valid directory"; - public static final String E_INSTALL_PATH_DOES_NOT_EXIST = "Install path does not exist"; - public static final String E_INVALID_APPLICATION_TYPE_NAME - = "A valid application type name is required (e.g. HBASE)."; - public static final String E_USE_REPLACEPKG_TO_OVERWRITE = "Use --replacepkg to overwrite."; - public static final String E_PACKAGE_DOES_NOT_EXIST = "Package does not exist"; - public static final String E_NO_ZOOKEEPER_QUORUM = "No Zookeeper quorum defined"; - public static final String E_NO_RESOURCE_MANAGER = "No valid Resource Manager address provided"; - public static final String E_PACKAGE_EXISTS = "Package exists"; - private static PrintStream clientOutputStream = System.out; - private static final JsonSerDeser jsonSerDeser = - new JsonSerDeser(Application.class, - PropertyNamingStrategy.CAMEL_CASE_TO_LOWER_CASE_WITH_UNDERSCORES); - - // value should not be changed without updating string find in slider.py - private static final String PASSWORD_PROMPT = "Enter password for"; - - private ClientArgs serviceArgs; - public ApplicationId applicationId; - - private String deployedClusterName; - /** - * Cluster operations against the deployed cluster -will be null - * if no bonding has yet taken place - */ - private SliderClusterOperations sliderClusterOperations; - - protected SliderFileSystem sliderFileSystem; - private YarnRPC rpc; - /** - * Yarn client service - */ - private SliderYarnClientImpl yarnClient; - private YarnAppListClient yarnAppListClient; - ResourceCalculator calculator; - /** - * The YARN registry service - */ - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private RegistryOperations registryOperations; - - private static EnumSet terminatedStates = - EnumSet.of(FINISHED, FAILED, KILLED); - private static EnumSet waitingStates = - EnumSet.of(NEW, NEW_SAVING, SUBMITTED, RUNNING); - - /** - * Constructor - */ - public SliderClient() { - super("Slider Client"); - new HdfsConfiguration(); - new YarnConfiguration(); - } - - /** - * This is called Before serviceInit is called - * @param config the initial configuration build up by the - * service launcher. - * @param args argument list list of arguments passed to the command line - * after any launcher-specific commands have been stripped. - * @return the post-binding configuration to pass to the init() - * operation. - * @throws Exception - */ - @Override - public Configuration bindArgs(Configuration config, String... args) throws Exception { - config = super.bindArgs(config, args); - serviceArgs = new ClientArgs(args); - serviceArgs.parse(); - // yarn-ify - YarnConfiguration yarnConfiguration = new YarnConfiguration(config); - return patchConfiguration(yarnConfiguration); - } - - @Override - protected void serviceInit(Configuration conf) throws Exception { - Configuration clientConf = loadSliderClientXML(); - ConfigHelper.mergeConfigurations(conf, clientConf, SLIDER_CLIENT_XML, true); - serviceArgs.applyDefinitions(conf); - serviceArgs.applyFileSystemBinding(conf); - AbstractActionArgs coreAction = serviceArgs.getCoreAction(); - // init security with our conf - if (!coreAction.disableSecureLogin() && isHadoopClusterSecure(conf)) { - forceLogin(); - initProcessSecurity(conf); - } - if (coreAction.getHadoopServicesRequired()) { - initHadoopBinding(); - } - rpc = YarnRPC.create(conf); - super.serviceInit(conf); - } - - @Override - protected void serviceStart() throws Exception { - super.serviceStart(); - } - - @Override - protected void serviceStop() throws Exception { - super.serviceStop(); - } - - /** - * Launched service execution. This runs {@link #exec()} - * then catches some exceptions and converts them to exit codes - * @return an exit code - * @throws Throwable - */ - @Override - public int runService() throws Throwable { - try { - return exec(); - } catch (FileNotFoundException | PathNotFoundException nfe) { - throw new NotFoundException(nfe, nfe.toString()); - } - } - - /** - * Execute the command line - * @return an exit code - * @throws Throwable on a failure - */ - public int exec() throws Throwable { - - // choose the action - String action = serviceArgs.getAction(); - if (isUnset(action)) { - throw new SliderException(EXIT_USAGE, serviceArgs.usage()); - } - - int exitCode = EXIT_SUCCESS; - String clusterName = serviceArgs.getClusterName(); - // actions - - switch (action) { - case ACTION_AM_SUICIDE: - exitCode = actionAmSuicide(clusterName, - serviceArgs.getActionAMSuicideArgs()); - break; - - case ACTION_BUILD: - exitCode = actionBuild(getApplicationFromArgs(clusterName, - serviceArgs.getActionBuildArgs())); - break; - - case ACTION_CLIENT: - exitCode = actionClient(serviceArgs.getActionClientArgs()); - break; - - case ACTION_CREATE: - actionCreate(getApplicationFromArgs(clusterName, - serviceArgs.getActionCreateArgs())); - break; - - case ACTION_DEPENDENCY: - exitCode = actionDependency(serviceArgs.getActionDependencyArgs()); - break; - - case ACTION_DESTROY: - actionDestroy(clusterName); - break; - - case ACTION_DIAGNOSTICS: - exitCode = actionDiagnostic(serviceArgs.getActionDiagnosticArgs()); - break; - - case ACTION_EXISTS: - exitCode = actionExists(clusterName, - serviceArgs.getActionExistsArgs()); - break; - - case ACTION_FLEX: - actionFlex(clusterName, serviceArgs.getActionFlexArgs()); - break; - - case ACTION_STOP: - actionStop(clusterName, serviceArgs.getActionFreezeArgs()); - break; - - case ACTION_HELP: - log.info(serviceArgs.usage()); - break; - - case ACTION_KDIAG: - exitCode = actionKDiag(serviceArgs.getActionKDiagArgs()); - break; - - case ACTION_KILL_CONTAINER: - exitCode = actionKillContainer(clusterName, - serviceArgs.getActionKillContainerArgs()); - break; - - case ACTION_KEYTAB: - exitCode = actionKeytab(serviceArgs.getActionKeytabArgs()); - break; - - case ACTION_LIST: - exitCode = actionList(clusterName, serviceArgs.getActionListArgs()); - break; - - case ACTION_LOOKUP: - exitCode = actionLookup(serviceArgs.getActionLookupArgs()); - break; - - case ACTION_NODES: - exitCode = actionNodes("", serviceArgs.getActionNodesArgs()); - break; - - case ACTION_REGISTRY: - exitCode = actionRegistry(serviceArgs.getActionRegistryArgs()); - break; - - case ACTION_RESOLVE: - exitCode = actionResolve(serviceArgs.getActionResolveArgs()); - break; - - case ACTION_RESOURCE: - exitCode = actionResource(serviceArgs.getActionResourceArgs()); - break; - - case ACTION_STATUS: - exitCode = actionStatus(clusterName, serviceArgs.getActionStatusArgs()); - break; - - case ACTION_START: - exitCode = actionStart(clusterName, serviceArgs.getActionThawArgs()); - break; - - case ACTION_TOKENS: - exitCode = actionTokens(serviceArgs.getActionTokenArgs()); - break; - - case ACTION_UPDATE: - exitCode = actionUpdate(clusterName, serviceArgs.getActionUpdateArgs()); - break; - - case ACTION_UPGRADE: - exitCode = actionUpgrade(clusterName, serviceArgs.getActionUpgradeArgs()); - break; - - case ACTION_VERSION: - exitCode = actionVersion(); - break; - - default: - throw new SliderException(EXIT_UNIMPLEMENTED, - "Unimplemented: " + action); - } - - return exitCode; - } - - /** - * Perform everything needed to init the hadoop binding. - * This assumes that the service is already in inited or started state - * @throws IOException - * @throws SliderException - */ - protected void initHadoopBinding() throws IOException, SliderException { - // validate the client - validateSliderClientEnvironment(null); - //create the YARN client - yarnClient = new SliderYarnClientImpl(); - yarnClient.init(getConfig()); - if (getServiceState() == STATE.STARTED) { - yarnClient.start(); - } - addService(yarnClient); - yarnAppListClient = - new YarnAppListClient(yarnClient, getUsername(), getConfig()); - // create the filesystem - sliderFileSystem = new SliderFileSystem(getConfig()); - } - - /** - * Delete the zookeeper node associated with the calling user and the cluster - * TODO: YARN registry operations - **/ - @VisibleForTesting - public boolean deleteZookeeperNode(String clusterName) throws YarnException, IOException { - String user = getUsername(); - String zkPath = ZKIntegration.mkClusterPath(user, clusterName); - Exception e = null; - try { - ZKIntegration client = getZkClient(clusterName, user); - if (client != null) { - if (client.exists(zkPath)) { - log.info("Deleting zookeeper path {}", zkPath); - } - client.deleteRecursive(zkPath); - return true; - } - } catch (InterruptedException | BadConfigException | KeeperException ex) { - e = ex; - } - if (e != null) { - log.warn("Unable to recursively delete zk node {}", zkPath, e); - } - - return false; - } - - /** - * Create the zookeeper node associated with the calling user and the cluster - * - * @param clusterName slider application name - * @param nameOnly should the name only be created (i.e. don't create ZK node) - * @return the path, using the policy implemented in - * {@link ZKIntegration#mkClusterPath(String, String)} - * @throws YarnException - * @throws IOException - */ - @VisibleForTesting - public String createZookeeperNode(String clusterName, Boolean nameOnly) throws YarnException, IOException { - try { - return createZookeeperNodeInner(clusterName, nameOnly); - } catch (KeeperException.NodeExistsException e) { - return null; - } catch (KeeperException e) { - return null; - } catch (InterruptedException e) { - throw new InterruptedIOException(e.toString()); - } - } - - /** - * Create the zookeeper node associated with the calling user and the cluster - * -throwing exceptions on any failure - * @param clusterName cluster name - * @param nameOnly create the path, not the node - * @return the path, using the policy implemented in - * {@link ZKIntegration#mkClusterPath(String, String)} - * @throws YarnException - * @throws IOException - * @throws KeeperException - * @throws InterruptedException - */ - @VisibleForTesting - public String createZookeeperNodeInner(String clusterName, Boolean nameOnly) - throws YarnException, IOException, KeeperException, InterruptedException { - String user = getUsername(); - String zkPath = ZKIntegration.mkClusterPath(user, clusterName); - if (nameOnly) { - return zkPath; - } - ZKIntegration client = getZkClient(clusterName, user); - if (client != null) { - // set up the permissions. This must be done differently on a secure cluster from an insecure - // one - List zkperms = new ArrayList<>(); - if (UserGroupInformation.isSecurityEnabled()) { - zkperms.add(new ACL(ZooDefs.Perms.ALL, ZooDefs.Ids.AUTH_IDS)); - zkperms.add(new ACL(ZooDefs.Perms.READ, ZooDefs.Ids.ANYONE_ID_UNSAFE)); - } else { - zkperms.add(new ACL(ZooDefs.Perms.ALL, ZooDefs.Ids.ANYONE_ID_UNSAFE)); - } - client.createPath(zkPath, "", - zkperms, - CreateMode.PERSISTENT); - return zkPath; - } else { - return null; - } - } - - /** - * Gets a zookeeper client, returns null if it cannot connect to zookeeper - **/ - protected ZKIntegration getZkClient(String clusterName, String user) throws YarnException { - String registryQuorum = lookupZKQuorum(); - ZKIntegration client = null; - try { - BlockingZKWatcher watcher = new BlockingZKWatcher(); - client = ZKIntegration.newInstance(registryQuorum, user, clusterName, true, false, watcher, - ZKIntegration.SESSION_TIMEOUT); - boolean fromCache = client.init(); - if (!fromCache) { - watcher.waitForZKConnection(2 * 1000); - } - } catch (InterruptedException e) { - client = null; - log.warn("Interrupted - unable to connect to zookeeper quorum {}", - registryQuorum, e); - } catch (IOException e) { - log.warn("Unable to connect to zookeeper quorum {}", registryQuorum, e); - } - return client; - } - - /** - * Keep this signature for backward compatibility with - * force=true by default. - */ - @Override - public int actionDestroy(String appName) - throws YarnException, IOException { - validateClusterName(appName); - verifyNoLiveApp(appName, "Destroy"); - Path appDir = sliderFileSystem.buildClusterDirPath(appName); - FileSystem fs = sliderFileSystem.getFileSystem(); - if (fs.exists(appDir)) { - if (fs.delete(appDir, true)) { - log.info("Successfully deleted application dir for " + appName); - } else { - String message = - "Failed to delete application + " + appName + " at: " + appDir; - log.info(message); - throw new YarnException(message); - } - } - if (!deleteZookeeperNode(appName)) { - String message = - "Failed to cleanup cleanup application " + appName + " in zookeeper"; - log.warn(message); - throw new YarnException(message); - } - - //TODO clean registry? - String registryPath = SliderRegistryUtils.registryPathForInstance( - appName); - try { - getRegistryOperations().delete(registryPath, true); - } catch (IOException e) { - log.warn("Error deleting registry entry {}: {} ", registryPath, e, e); - } catch (SliderException e) { - log.warn("Error binding to registry {} ", e, e); - } - - log.info("Destroyed cluster {}", appName); - return EXIT_SUCCESS; - } - - - @Override - public int actionAmSuicide(String clustername, - ActionAMSuicideArgs args) throws YarnException, IOException { - SliderClusterOperations clusterOperations = - createClusterOperations(clustername); - clusterOperations.amSuicide(args.message, args.exitcode, args.waittime); - return EXIT_SUCCESS; - } - - private Application getApplicationFromArgs(String clusterName, - AbstractClusterBuildingActionArgs args) throws IOException { - File file = args.getAppDef(); - Path filePath = new Path(file.getAbsolutePath()); - log.info("Loading app definition from: " + filePath); - Application application = - jsonSerDeser.load(FileSystem.getLocal(getConfig()), filePath); - if(args.lifetime > 0) { - application.setLifetime(args.lifetime); - } - application.setName(clusterName); - return application; - } - - public int actionBuild(Application application) throws YarnException, - IOException { - Path appDir = checkAppNotExistOnHdfs(application); - ServiceApiUtil.validateAndResolveApplication(application, - sliderFileSystem, getConfig()); - persistApp(appDir, application); - deployedClusterName = application.getName(); - return EXIT_SUCCESS; - } - - public ApplicationId actionCreate(Application application) - throws IOException, YarnException { - String appName = application.getName(); - validateClusterName(appName); - ServiceApiUtil.validateAndResolveApplication(application, - sliderFileSystem, getConfig()); - verifyNoLiveApp(appName, "Create"); - Path appDir = checkAppNotExistOnHdfs(application); - - ApplicationId appId = submitApp(application); - application.setId(appId.toString()); - // write app definition on to hdfs - persistApp(appDir, application); - return appId; - //TODO deal with registry - } - - private ApplicationId submitApp(Application app) - throws IOException, YarnException { - String appName = app.getName(); - Configuration conf = getConfig(); - Path appRootDir = sliderFileSystem.buildClusterDirPath(app.getName()); - deployedClusterName = appName; - - YarnClientApplication yarnApp = yarnClient.createApplication(); - ApplicationSubmissionContext submissionContext = - yarnApp.getApplicationSubmissionContext(); - ServiceApiUtil.validateCompResourceSize( - yarnApp.getNewApplicationResponse().getMaximumResourceCapability(), - app); - - applicationId = submissionContext.getApplicationId(); - submissionContext.setKeepContainersAcrossApplicationAttempts(true); - if (app.getLifetime() > 0) { - Map appTimeout = new HashMap<>(); - appTimeout.put(ApplicationTimeoutType.LIFETIME, app.getLifetime()); - submissionContext.setApplicationTimeouts(appTimeout); - } - submissionContext.setMaxAppAttempts(conf.getInt(KEY_AM_RESTART_LIMIT, 2)); - - Map localResources = new HashMap<>(); - - // copy local slideram-log4j.properties to hdfs and add to localResources - boolean hasSliderAMLog4j = - addAMLog4jResource(appName, conf, localResources); - // copy jars to hdfs and add to localResources - addJarResource(appName, localResources); - // add keytab if in secure env - addKeytabResourceIfSecure(sliderFileSystem, localResources, conf, appName); - printLocalResources(localResources); - - //TODO SliderAMClientProvider#copyEnvVars - //TODO localResource putEnv - - Map env = addAMEnv(conf); - - // create AM CLI - String cmdStr = - buildCommandLine(appName, conf, appRootDir, hasSliderAMLog4j); - - //TODO set log aggregation context - //TODO set retry window - submissionContext.setResource(Resource.newInstance( - conf.getLong(KEY_AM_RESOURCE_MEM, DEFAULT_KEY_AM_RESOURCE_MEM), 1)); - submissionContext.setQueue(conf.get(KEY_YARN_QUEUE, app.getQueue())); - submissionContext.setApplicationName(appName); - submissionContext.setApplicationType(SliderKeys.APP_TYPE); - Set appTags = - AbstractClientProvider.createApplicationTags(appName, 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); - addCredentialsIfSecure(conf, amLaunchContext); - submissionContext.setAMContainerSpec(amLaunchContext); - submitApplication(submissionContext); - return submissionContext.getApplicationId(); - } - - @VisibleForTesting - public ApplicationId submitApplication(ApplicationSubmissionContext context) - throws IOException, YarnException { - return yarnClient.submitApplication(context); - } - - private void printLocalResources(Map map) { - log.info("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.info(builder.toString()); - } - - private void addCredentialsIfSecure(Configuration conf, - ContainerLaunchContext amLaunchContext) throws IOException { - if (UserGroupInformation.isSecurityEnabled()) { - // pick up oozie credentials - Credentials credentials = - CredentialUtils.loadTokensFromEnvironment(System.getenv(), conf); - if (credentials == null) { - // nothing from oozie, so build up directly - credentials = new Credentials( - UserGroupInformation.getCurrentUser().getCredentials()); - CredentialUtils.addRMRenewableFSDelegationTokens(conf, - sliderFileSystem.getFileSystem(), credentials); - } else { - log.info("Using externally supplied credentials to launch AM"); - } - amLaunchContext.setTokens(CredentialUtils.marshallCredentials(credentials)); - } - } - - private String buildCommandLine(String appName, 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, LOG4J_SERVER_PROP_FILENAME); - CLI.sysprop(SYSPROP_LOG_DIR, ApplicationConstants.LOG_DIR_EXPANSION_VAR); - } - CLI.add(ServiceMaster.class.getCanonicalName()); - CLI.add(ACTION_CREATE, appName); - //TODO debugAM CLI.add(Arguments.ARG_DEBUG) - CLI.add(Arguments.ARG_CLUSTER_URI, new Path(appRootDir, appName + ".json")); - // InetSocketAddress rmSchedulerAddress = getRmSchedulerAddress(conf); -// String rmAddr = NetUtils.getHostPortString(rmSchedulerAddress); -// CLI.add(Arguments.ARG_RM_ADDR, rmAddr); - // 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); - if(isHadoopClusterSecure(conf)) { - //TODO Is this required ?? - // if the cluster is secure, make sure that - // the relevant security settings go over - CLI.addConfOption(conf, DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY); - } -// // copy over any/all YARN RM client values, in case the server-side XML conf file -// // has the 0.0.0.0 address -// CLI.addConfOptions(conf, YarnConfiguration.RM_ADDRESS, -// YarnConfiguration.RM_CLUSTER_ID, YarnConfiguration.RM_HOSTNAME, -// YarnConfiguration.RM_PRINCIPAL); - - // write out the path output - CLI.addOutAndErrFiles(STDOUT_AM, STDERR_AM); - String cmdStr = CLI.build(); - log.info("Completed setting up app master command: {}", cmdStr); - return cmdStr; - } - - private Map addAMEnv(Configuration conf) - throws IOException { - Map env = new HashMap<>(); - ClasspathConstructor classpath = - buildClasspath(SliderKeys.SUBMITTED_CONF_DIR, "lib", - sliderFileSystem, getUsingMiniMRCluster()); - 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); - } - env.putAll(getAmLaunchEnv(conf)); - log.info("AM env: \n{}", stringifyMap(env)); - return env; - } - - protected Path addJarResource(String appName, - Map localResources) - throws IOException, SliderException { - Path libPath = sliderFileSystem.buildClusterDirPath(appName); - ProviderUtils - .addProviderJar(localResources, ServiceMaster.class, SLIDER_JAR, - sliderFileSystem, libPath, "lib", false); - Path dependencyLibTarGzip = sliderFileSystem.getDependencyTarGzip(); - if (sliderFileSystem.isFile(dependencyLibTarGzip)) { - log.info("Loading lib tar from " + sliderFileSystem.getFileSystem() - .getScheme() + ": " + dependencyLibTarGzip); - SliderUtils.putAmTarGzipAndUpdate(localResources, sliderFileSystem); - } else { - String[] libs = SliderUtils.getLibDirs(); - log.info("Loading dependencies from local file system: " + Arrays - .toString(libs)); - for (String libDirProp : libs) { - ProviderUtils - .addAllDependencyJars(localResources, sliderFileSystem, libPath, - "lib", libDirProp); - } - } - return libPath; - } - - private boolean addAMLog4jResource(String appName, Configuration conf, - Map localResources) - throws IOException, BadClusterStateException { - boolean hasSliderAMLog4j = false; - String hadoopConfDir = - System.getenv(ApplicationConstants.Environment.HADOOP_CONF_DIR.name()); - if (hadoopConfDir != null) { - File localFile = - new File(hadoopConfDir, SliderKeys.LOG4J_SERVER_PROP_FILENAME); - if (localFile.exists()) { - Path localFilePath = createLocalPath(localFile); - Path appDirPath = sliderFileSystem.buildClusterDirPath(appName); - Path remoteConfPath = - new Path(appDirPath, SliderKeys.SUBMITTED_CONF_DIR); - Path remoteFilePath = - new Path(remoteConfPath, SliderKeys.LOG4J_SERVER_PROP_FILENAME); - copy(conf, localFilePath, remoteFilePath); - LocalResource localResource = sliderFileSystem - .createAmResource(remoteConfPath, LocalResourceType.FILE); - localResources.put(localFilePath.getName(), localResource); - hasSliderAMLog4j = true; - } - } - return hasSliderAMLog4j; - } - - private Path checkAppNotExistOnHdfs(Application application) - throws IOException, SliderException { - Path appDir = sliderFileSystem.buildClusterDirPath(application.getName()); - sliderFileSystem.verifyDirectoryNonexistent( - new Path(appDir, application.getName() + ".json")); - return appDir; - } - - private Path checkAppExistOnHdfs(String appName) - throws IOException, SliderException { - Path appDir = sliderFileSystem.buildClusterDirPath(appName); - sliderFileSystem.verifyPathExists( - new Path(appDir, appName + ".json")); - return appDir; - } - - private void persistApp(Path appDir, Application application) - throws IOException, SliderException { - FsPermission appDirPermission = new FsPermission("750"); - sliderFileSystem.createWithPermissions(appDir, appDirPermission); - Path appJson = new Path(appDir, application.getName() + ".json"); - jsonSerDeser - .save(sliderFileSystem.getFileSystem(), appJson, application, true); - log.info( - "Persisted application " + application.getName() + " at " + appJson); - } - - private void addKeytabResourceIfSecure(SliderFileSystem fileSystem, - Map localResource, Configuration conf, - String appName) throws IOException, BadConfigException { - if (!UserGroupInformation.isSecurityEnabled()) { - return; - } - String keytabPreInstalledOnHost = - conf.get(SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH); - if (StringUtils.isEmpty(keytabPreInstalledOnHost)) { - String amKeytabName = - conf.get(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME); - String keytabDir = conf.get(SliderXmlConfKeys.KEY_HDFS_KEYTAB_DIR); - Path keytabPath = - fileSystem.buildKeytabPath(keytabDir, amKeytabName, appName); - if (fileSystem.getFileSystem().exists(keytabPath)) { - LocalResource keytabRes = - fileSystem.createAmResource(keytabPath, LocalResourceType.FILE); - localResource - .put(SliderKeys.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(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 application is therefore not guaranteed to remain " - + "operational beyond 24 hours."); - } - } - } - } - - @Override - public int actionUpgrade(String clustername, ActionUpgradeArgs upgradeArgs) - throws YarnException, IOException { - //TODO - return 0; - } - - @Override - public int actionKeytab(ActionKeytabArgs keytabInfo) - throws YarnException, IOException { - if (keytabInfo.install) { - return actionInstallKeytab(keytabInfo); - } else if (keytabInfo.delete) { - return actionDeleteKeytab(keytabInfo); - } else if (keytabInfo.list) { - return actionListKeytab(keytabInfo); - } else { - throw new BadCommandArgumentsException( - "Keytab option specified not found.\n" - + CommonArgs.usage(serviceArgs, ACTION_KEYTAB)); - } - } - - private int actionListKeytab(ActionKeytabArgs keytabInfo) throws IOException { - String folder = keytabInfo.folder != null ? keytabInfo.folder : StringUtils.EMPTY; - Path keytabPath = sliderFileSystem.buildKeytabInstallationDirPath(folder); - RemoteIterator files = - sliderFileSystem.getFileSystem().listFiles(keytabPath, true); - log.info("Keytabs:"); - while (files.hasNext()) { - log.info("\t" + files.next().getPath().toString()); - } - - return EXIT_SUCCESS; - } - - private int actionDeleteKeytab(ActionKeytabArgs keytabInfo) - throws BadCommandArgumentsException, IOException { - if (StringUtils.isEmpty(keytabInfo.folder)) { - throw new BadCommandArgumentsException( - "A valid destination keytab sub-folder name is required (e.g. 'security').\n" - + CommonArgs.usage(serviceArgs, ACTION_KEYTAB)); - } - - if (StringUtils.isEmpty(keytabInfo.keytab)) { - throw new BadCommandArgumentsException("A keytab name is required."); - } - - Path pkgPath = sliderFileSystem.buildKeytabInstallationDirPath(keytabInfo.folder); - - Path fileInFs = new Path(pkgPath, keytabInfo.keytab ); - log.info("Deleting keytab {}", fileInFs); - FileSystem sfs = sliderFileSystem.getFileSystem(); - require(sfs.exists(fileInFs), "No keytab to delete found at %s", - fileInFs.toUri()); - sfs.delete(fileInFs, false); - - return EXIT_SUCCESS; - } - - private int actionInstallKeytab(ActionKeytabArgs keytabInfo) - throws BadCommandArgumentsException, IOException { - Path srcFile = null; - require(isSet(keytabInfo.folder), - "A valid destination keytab sub-folder name is required (e.g. 'security').\n" - + CommonArgs.usage(serviceArgs, ACTION_KEYTAB)); - - requireArgumentSet(Arguments.ARG_KEYTAB, keytabInfo.keytab); - File keytabFile = new File(keytabInfo.keytab); - require(keytabFile.isFile(), - "Unable to access supplied keytab file at %s", keytabFile.getAbsolutePath()); - srcFile = new Path(keytabFile.toURI()); - - Path pkgPath = sliderFileSystem.buildKeytabInstallationDirPath(keytabInfo.folder); - FileSystem sfs = sliderFileSystem.getFileSystem(); - sfs.mkdirs(pkgPath); - sfs.setPermission(pkgPath, new FsPermission( - FsAction.ALL, FsAction.NONE, FsAction.NONE)); - - Path fileInFs = new Path(pkgPath, srcFile.getName()); - log.info("Installing keytab {} at {} and overwrite is {}.", - srcFile, fileInFs, keytabInfo.overwrite); - require(!(sfs.exists(fileInFs) && !keytabInfo.overwrite), - "Keytab exists at %s. Use --overwrite to overwrite.", fileInFs.toUri()); - - sfs.copyFromLocalFile(false, keytabInfo.overwrite, srcFile, fileInFs); - sfs.setPermission(fileInFs, - new FsPermission(FsAction.READ_WRITE, FsAction.NONE, FsAction.NONE)); - - return EXIT_SUCCESS; - } - - @Override - public int actionResource(ActionResourceArgs resourceInfo) - throws YarnException, IOException { - if (resourceInfo.help) { - actionHelp(ACTION_RESOURCE); - return EXIT_SUCCESS; - } else if (resourceInfo.install) { - return actionInstallResource(resourceInfo); - } else if (resourceInfo.delete) { - return actionDeleteResource(resourceInfo); - } else if (resourceInfo.list) { - return actionListResource(resourceInfo); - } else { - throw new BadCommandArgumentsException( - "Resource option specified not found.\n" - + CommonArgs.usage(serviceArgs, ACTION_RESOURCE)); - } - } - - private int actionListResource(ActionResourceArgs resourceInfo) throws IOException { - String folder = resourceInfo.folder != null ? resourceInfo.folder : StringUtils.EMPTY; - Path path = sliderFileSystem.buildResourcePath(folder); - RemoteIterator files = - sliderFileSystem.getFileSystem().listFiles(path, true); - log.info("Resources:"); - while (files.hasNext()) { - log.info("\t" + files.next().getPath().toString()); - } - - return EXIT_SUCCESS; - } - - private int actionDeleteResource(ActionResourceArgs resourceInfo) - throws BadCommandArgumentsException, IOException { - if (StringUtils.isEmpty(resourceInfo.resource)) { - throw new BadCommandArgumentsException("A file name is required."); - } - - Path fileInFs; - if (resourceInfo.folder == null) { - fileInFs = sliderFileSystem.buildResourcePath(resourceInfo.resource); - } else { - fileInFs = sliderFileSystem.buildResourcePath(resourceInfo.folder, - resourceInfo.resource); - } - - log.info("Deleting resource {}", fileInFs); - FileSystem sfs = sliderFileSystem.getFileSystem(); - require(sfs.exists(fileInFs), "No resource to delete found at %s", fileInFs.toUri()); - sfs.delete(fileInFs, true); - - return EXIT_SUCCESS; - } - - private int actionInstallResource(ActionResourceArgs resourceInfo) - throws BadCommandArgumentsException, IOException { - Path srcFile = null; - String folder = resourceInfo.folder != null ? resourceInfo.folder : StringUtils.EMPTY; - - requireArgumentSet(Arguments.ARG_RESOURCE, resourceInfo.resource); - File file = new File(resourceInfo.resource); - require(file.isFile() || file.isDirectory(), - "Unable to access supplied file at %s", file.getAbsolutePath()); - - File[] files; - if (file.isDirectory()) { - files = file.listFiles(); - } else { - files = new File[] { file }; - } - - Path pkgPath = sliderFileSystem.buildResourcePath(folder); - FileSystem sfs = sliderFileSystem.getFileSystem(); - - if (!sfs.exists(pkgPath)) { - sfs.mkdirs(pkgPath); - sfs.setPermission(pkgPath, new FsPermission( - FsAction.ALL, FsAction.NONE, FsAction.NONE)); - } else { - require(sfs.isDirectory(pkgPath), "Specified folder %s exists and is " + - "not a directory", folder); - } - - if (files != null) { - for (File f : files) { - srcFile = new Path(f.toURI()); - - Path fileInFs = new Path(pkgPath, srcFile.getName()); - log.info("Installing file {} at {} and overwrite is {}.", - srcFile, fileInFs, resourceInfo.overwrite); - require(!(sfs.exists(fileInFs) && !resourceInfo.overwrite), - "File exists at %s. Use --overwrite to overwrite.", fileInFs.toUri()); - - sfs.copyFromLocalFile(false, resourceInfo.overwrite, srcFile, fileInFs); - sfs.setPermission(fileInFs, - new FsPermission(FsAction.READ_WRITE, FsAction.NONE, FsAction.NONE)); - } - } - - return EXIT_SUCCESS; - } - - @Override - public int actionClient(ActionClientArgs clientInfo) throws - YarnException, - IOException { - if (clientInfo.install) { - // TODO implement client install - throw new UnsupportedOperationException("Client install not yet " + - "supported"); - } else { - throw new BadCommandArgumentsException( - "Only install, keystore, and truststore commands are supported for the client.\n" - + CommonArgs.usage(serviceArgs, ACTION_CLIENT)); - - } - } - - @Override - public int actionUpdate(String clustername, - AbstractClusterBuildingActionArgs buildInfo) throws - YarnException, IOException { - if (buildInfo.lifetime > 0) { - updateLifetime(clustername, buildInfo.lifetime); - } else { - //TODO upgrade - } - return EXIT_SUCCESS; - } - - public String updateLifetime(String appName, long lifetime) - throws YarnException, IOException { - EnumSet appStates = EnumSet.range(NEW, RUNNING); - ApplicationReport report = findInstance(appName, appStates); - if (report == null) { - throw new YarnException("Application not found for " + appName); - } - ApplicationId appId = report.getApplicationId(); - log.info("Updating lifetime of an application: appName = " + appName - + ", 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 application: appName = " - + appName + ", appId = " + appId - + ". New expiry time in ISO8601 format is " + newTimeout); - return newTimeout; - } - - protected Map getAmLaunchEnv(Configuration config) { - String sliderAmLaunchEnv = config.get(KEY_AM_LAUNCH_ENV); - log.debug("{} = {}", KEY_AM_LAUNCH_ENV, sliderAmLaunchEnv); - // Multiple env variables can be specified with a comma (,) separator - String[] envs = StringUtils.isEmpty(sliderAmLaunchEnv) ? null - : sliderAmLaunchEnv.split(","); - if (ArrayUtils.isEmpty(envs)) { - return Collections.emptyMap(); - } - Map amLaunchEnv = new HashMap<>(); - for (String env : envs) { - if (StringUtils.isNotEmpty(env)) { - // Each env name/value is separated by equals sign (=) - String[] tokens = env.split("="); - if (tokens != null && tokens.length == 2) { - String envKey = tokens[0]; - String envValue = tokens[1]; - for (Map.Entry placeholder : generatePlaceholderKeyValueMap( - env).entrySet()) { - if (StringUtils.isNotEmpty(placeholder.getValue())) { - envValue = envValue.replaceAll( - Pattern.quote(placeholder.getKey()), placeholder.getValue()); - } - } - if (Shell.WINDOWS) { - envValue = "%" + envKey + "%;" + envValue; - } else { - envValue = "$" + envKey + ":" + envValue; - } - log.info("Setting AM launch env {}={}", envKey, envValue); - amLaunchEnv.put(envKey, envValue); - } - } - } - return amLaunchEnv; - } - - protected Map generatePlaceholderKeyValueMap(String env) { - String PLACEHOLDER_PATTERN = "\\$\\{[^{]+\\}"; - Pattern placeholderPattern = Pattern.compile(PLACEHOLDER_PATTERN); - Matcher placeholderMatcher = placeholderPattern.matcher(env); - Map placeholderKeyValueMap = new HashMap<>(); - if (placeholderMatcher.find()) { - String placeholderKey = placeholderMatcher.group(); - String systemKey = placeholderKey - .substring(2, placeholderKey.length() - 1).toUpperCase(Locale.ENGLISH) - .replaceAll("\\.", "_"); - String placeholderValue = getSystemEnv(systemKey); - log.debug("Placeholder {}={}", placeholderKey, placeholderValue); - placeholderKeyValueMap.put(placeholderKey, placeholderValue); - } - return placeholderKeyValueMap; - } - - /** - * verify that a live cluster isn't there - * @param clustername cluster name - * @param action - * @throws SliderException with exit code EXIT_CLUSTER_LIVE - * if a cluster of that name is either live or starting up. - */ - public void verifyNoLiveApp(String clustername, String action) throws - IOException, - YarnException { - List existing = findAllLiveInstances(clustername); - - if (!existing.isEmpty()) { - throw new SliderException(EXIT_APPLICATION_IN_USE, - action +" failed for " - + clustername - + ": " - + E_CLUSTER_RUNNING + " :" + - existing.get(0)); - } - } - - public String getUsername() throws IOException { - return RegistryUtils.currentUser(); - } - - /** - * Get the name of any deployed cluster - * @return the cluster name - */ - public String getDeployedClusterName() { - return deployedClusterName; - } - - /** - * ask if the client is using a mini MR cluster - * @return true if they are - */ - private boolean getUsingMiniMRCluster() { - return getConfig().getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, - false); - } - - - /** - * List Slider instances belonging to a specific user with a specific app - * name and within a set of app states. - * @param user user: "" means all users, null means "default" - * @param appName name of the application set as a tag - * @param appStates a set of states the applications should be in - * @return a possibly empty list of Slider AMs - */ - public List listSliderInstances(String user, - String appName, EnumSet appStates) - throws YarnException, IOException { - return yarnAppListClient.listInstances(user, appName, appStates); - } - - /** - * A basic list action to list live instances - * @param clustername cluster name - * @return success if the listing was considered successful - * @throws IOException - * @throws YarnException - */ - public int actionList(String clustername) throws IOException, YarnException { - ActionListArgs args = new ActionListArgs(); - args.live = true; - return actionList(clustername, args); - } - - /** - * Implement the list action. - * @param clustername List out specific instance name - * @param args Action list arguments - * @return 0 if one or more entries were listed - * @throws IOException - * @throws YarnException - * @throws UnknownApplicationInstanceException if a specific instance - * was named but it was not found - */ - @Override - public int actionList(String clustername, ActionListArgs args) - throws IOException, YarnException { - Set appInstances = getApplicationList(clustername, args); - if (!appInstances.isEmpty()) { - return EXIT_SUCCESS; - } else { - return EXIT_FALSE; - } - } - - /** - * Retrieve a list of application instances satisfying the query criteria. - * - * @param clustername - * List out specific instance name (set null for all) - * @param args - * Action list arguments - * @return the list of application names which satisfies the list criteria - * @throws IOException - * @throws YarnException - * @throws UnknownApplicationInstanceException - * if a specific instance was named but it was not found - */ - public Set getApplicationList(String clustername, - ActionListArgs args) throws IOException, YarnException { - if (args.help) { - actionHelp(ACTION_LIST); - // the above call throws an exception so the return is not really required - return Collections.emptySet(); - } - boolean live = args.live; - String state = args.state; - boolean listContainers = args.containers; - boolean verbose = args.verbose; - String version = args.version; - Set components = args.components; - - if (live && !state.isEmpty()) { - throw new BadCommandArgumentsException( - Arguments.ARG_LIVE + " and " + Arguments.ARG_STATE + " are exclusive"); - } - if (listContainers && isUnset(clustername)) { - throw new BadCommandArgumentsException( - "Should specify an application instance with " - + Arguments.ARG_CONTAINERS); - } - // specifying both --version and --components with --containers is okay - if (StringUtils.isNotEmpty(version) && !listContainers) { - throw new BadCommandArgumentsException(Arguments.ARG_VERSION - + " can be specified only with " + Arguments.ARG_CONTAINERS); - } - if (!components.isEmpty() && !listContainers) { - throw new BadCommandArgumentsException(Arguments.ARG_COMPONENTS - + " can be specified only with " + Arguments.ARG_CONTAINERS); - } - - // flag to indicate only services in a specific state are to be listed - boolean listOnlyInState = live || !state.isEmpty(); - - YarnApplicationState min, max; - if (live) { - min = NEW; - max = RUNNING; - } else if (!state.isEmpty()) { - YarnApplicationState stateVal = extractYarnApplicationState(state); - min = max = stateVal; - } else { - min = NEW; - max = KILLED; - } - // get the complete list of persistent instances - Map persistentInstances = sliderFileSystem.listPersistentInstances(); - - if (persistentInstances.isEmpty() && isUnset(clustername)) { - // an empty listing is a success if no cluster was named - log.debug("No application instances found"); - return Collections.emptySet(); - } - - // and those the RM knows about - EnumSet appStates = EnumSet.range(min, max); - List instances = listSliderInstances(null, clustername, - appStates); - sortApplicationsByMostRecent(instances); - Map reportMap = - buildApplicationReportMap(instances, min, max); - log.debug("Persisted {} deployed {} filtered[{}-{}] & de-duped to {}", - persistentInstances.size(), - instances.size(), - min, max, - reportMap.size() ); - - List containers = null; - if (isSet(clustername)) { - // only one instance is expected - // resolve the persistent value - Path persistent = persistentInstances.get(clustername); - if (persistent == null) { - throw unknownClusterException(clustername); - } - // create a new map with only that instance in it. - // this restricts the output of results to this instance - persistentInstances = new HashMap<>(); - persistentInstances.put(clustername, persistent); - if (listContainers) { - containers = getContainers(clustername); - } - } - - // at this point there is either the entire list or a stripped down instance - Set listedInstances = new HashSet(); - for (String name : persistentInstances.keySet()) { - ApplicationReport report = reportMap.get(name); - if (!listOnlyInState || report != null) { - // list the details if all were requested, or the filtering contained - // a report - listedInstances.add(report); - // containers will be non-null when only one instance is requested - String details = instanceDetailsToString(name, report, - containers, version, components, verbose); - print(details); - } - } - - return listedInstances; - } - - public List getContainers(String name) - throws YarnException, IOException { - SliderClusterOperations clusterOps = new SliderClusterOperations( - bondToCluster(name)); - try { - return clusterOps.getContainers(); - } catch (NoSuchNodeException e) { - throw new BadClusterStateException( - "Containers not found for application instance %s", name); - } - } - - - /** - * Extract the state of a Yarn application --state argument - * @param state state argument - * @return the application state - * @throws BadCommandArgumentsException if the argument did not match - * any known state - */ - private YarnApplicationState extractYarnApplicationState(String state) throws - BadCommandArgumentsException { - YarnApplicationState stateVal; - try { - stateVal = YarnApplicationState.valueOf(state.toUpperCase(Locale.ENGLISH)); - } catch (IllegalArgumentException e) { - throw new BadCommandArgumentsException("Unknown state: " + state); - - } - return stateVal; - } - - /** - * Is an application active: accepted or running - * @param report the application report - * @return true if it is running or scheduled to run. - */ - public boolean isApplicationActive(ApplicationReport report) { - return report.getYarnApplicationState() == RUNNING - || report.getYarnApplicationState() == YarnApplicationState.ACCEPTED; - } - - /** - * Implement the islive action: probe for a cluster of the given name existing - * @return exit code - */ - - @Override - @VisibleForTesting - public int actionFlex(String appName, ActionFlexArgs args) - throws YarnException, IOException { - Map componentCounts = new HashMap<>(args.getComponentMap() - .size()); - for (Entry entry : args.getComponentMap().entrySet()) { - long numberOfContainers = Long.parseLong(entry.getValue()); - componentCounts.put(entry.getKey(), numberOfContainers); - } - // throw usage exception if no changes proposed - if (componentCounts.size() == 0) { - actionHelp(ACTION_FLEX); - } - flex(appName, componentCounts); - return EXIT_SUCCESS; - } - - @Override - public int actionExists(String name, boolean checkLive) throws YarnException, IOException { - ActionExistsArgs args = new ActionExistsArgs(); - args.live = checkLive; - return actionExists(name, args); - } - - public int actionExists(String name, ActionExistsArgs args) throws YarnException, IOException { - validateClusterName(name); - boolean checkLive = args.live; - log.debug("actionExists({}, {}, {})", name, checkLive, args.state); - - //initial probe for a cluster in the filesystem - Path clusterDirectory = sliderFileSystem.buildClusterDirPath(name); - if (!sliderFileSystem.getFileSystem().exists(clusterDirectory)) { - throw unknownClusterException(name); - } - String state = args.state; - if (!checkLive && isUnset(state)) { - log.info("Application {} exists", name); - return EXIT_SUCCESS; - } - - //test for liveness/state - boolean inDesiredState = false; - ApplicationReport instance; - instance = findInstance(name); - if (instance == null) { - log.info("Application {} not running", name); - return EXIT_FALSE; - } - if (checkLive) { - // the app exists, check that it is not in any terminated state - YarnApplicationState appstate = instance.getYarnApplicationState(); - log.debug(" current app state = {}", appstate); - inDesiredState = appstate.ordinal() < FINISHED.ordinal(); - } else { - // scan for instance in single --state state - state = state.toUpperCase(Locale.ENGLISH); - YarnApplicationState desiredState = extractYarnApplicationState(state); - List userInstances = yarnClient - .listDeployedInstances("", EnumSet.of(desiredState), name); - ApplicationReport foundInstance = - yarnClient.findAppInInstanceList(userInstances, name, desiredState); - if (foundInstance != null) { - // found in selected state: success - inDesiredState = true; - // mark this as the instance to report - instance = foundInstance; - } - } - - OnDemandReportStringifier report = - new OnDemandReportStringifier(instance); - if (!inDesiredState) { - //cluster in the list of apps but not running - log.info("Application {} found but is in wrong state {}", name, - instance.getYarnApplicationState()); - log.debug("State {}", report); - return EXIT_FALSE; - } else { - log.debug("Application instance is in desired state"); - log.info("Application {} is {}\n{}", name, - instance.getYarnApplicationState(), report); - return EXIT_SUCCESS; - } - } - - - @Override - public int actionKillContainer(String name, - ActionKillContainerArgs args) throws YarnException, IOException { - String id = args.id; - if (isUnset(id)) { - throw new BadCommandArgumentsException("Missing container id"); - } - log.info("killingContainer {}:{}", name, id); - SliderClusterOperations clusterOps = - new SliderClusterOperations(bondToCluster(name)); - try { - clusterOps.killContainer(id); - } catch (NoSuchNodeException e) { - throw new BadClusterStateException("Container %s not found in cluster %s", - id, name); - } - return EXIT_SUCCESS; - } - - /** - * Find an instance of an application belonging to the current user. - * @param appname application name - * @return the app report or null if none is found - * @throws YarnException YARN issues - * @throws IOException IO problems - */ - public ApplicationReport findInstance(String appname) - throws YarnException, IOException { - return findInstance(appname, null); - } - - /** - * Find an instance of an application belonging to the current user and in - * specific app states. - * @param appname application name - * @param appStates app states in which the application should be in - * @return the app report or null if none is found - * @throws YarnException YARN issues - * @throws IOException IO problems - */ - public ApplicationReport findInstance(String appname, - EnumSet appStates) - throws YarnException, IOException { - return yarnAppListClient.findInstance(appname, appStates); - } - - /** - * find all live instances of a specific app -if there is >1 in the cluster, - * this returns them all. State should be running or less - * @param appname application name - * @return the list of all matching application instances - */ - private List findAllLiveInstances(String appname) - throws YarnException, IOException { - - return yarnAppListClient.findAllLiveInstances(appname); - } - - /** - * Connect to a Slider AM - * @param app application report providing the details on the application - * @return an instance - * @throws YarnException - * @throws IOException - */ - private SliderClusterProtocol connect(ApplicationReport app) - throws YarnException, IOException { - - try { - return RpcBinder.getProxy(getConfig(), - yarnClient.getRmClient(), - app, - Constants.CONNECT_TIMEOUT, - Constants.RPC_TIMEOUT); - } catch (InterruptedException e) { - throw new SliderException(SliderExitCodes.EXIT_TIMED_OUT, - e, - "Interrupted waiting for communications with the Slider AM"); - } - } - - @Override - @VisibleForTesting - public int actionStatus(String clustername, ActionStatusArgs statusArgs) - throws YarnException, IOException { - if (statusArgs.lifetime) { - queryAndPrintLifetime(clustername); - return EXIT_SUCCESS; - } - - Application application = getApplication(clustername); - String outfile = statusArgs.getOutput(); - if (outfile == null) { - log.info(application.toString()); - } else { - jsonSerDeser.save(application, new File(statusArgs.getOutput())); - } - return EXIT_SUCCESS; - } - - @Override - public Application actionStatus(String clustername) - throws YarnException, IOException { - return getApplication(clustername); - } - - private void queryAndPrintLifetime(String appName) - throws YarnException, IOException { - ApplicationReport appReport = findInstance(appName); - if (appReport == null) { - throw new YarnException("No application found for " + appName); - } - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - PrintWriter timeoutStr = - new PrintWriter(new OutputStreamWriter(baos, Charset.forName("UTF-8"))); - try { - ApplicationTimeout lifetime = appReport.getApplicationTimeouts() - .get(ApplicationTimeoutType.LIFETIME); - if (lifetime.getRemainingTime() == -1L) { - timeoutStr.append(appName + " has no lifetime configured."); - } else { - timeoutStr.append("\t" + ApplicationTimeoutType.LIFETIME); - timeoutStr.print(" expires at : " + lifetime.getExpiryTime()); - timeoutStr.println( - ".\tRemaining Time : " + lifetime.getRemainingTime() + " seconds"); - } - System.out.println(baos.toString("UTF-8")); - } finally { - timeoutStr.close(); - } - } - - @Override - public int actionVersion() { - SliderVersionInfo.loadAndPrintVersionInfo(log); - return EXIT_SUCCESS; - } - - @Override - public int actionStop(String appName, ActionFreezeArgs freezeArgs) - throws YarnException, IOException { - validateClusterName(appName); - ApplicationReport app = findInstance(appName); - if (app == null) { - throw new ApplicationNotFoundException( - "Application " + appName + " doesn't exist in RM."); - } - - if (terminatedStates.contains(app.getYarnApplicationState())) { - log.info("Application {} is already in a terminated state {}", appName, - app.getYarnApplicationState()); - return EXIT_SUCCESS; - } - - try { - SliderClusterProtocol appMaster = connect(app); - Messages.StopClusterRequestProto r = - Messages.StopClusterRequestProto.newBuilder() - .setMessage(freezeArgs.message).build(); - appMaster.stopCluster(r); - log.info("Application " + appName + " is being gracefully stopped..."); - long startTime = System.currentTimeMillis(); - int pollCount = 0; - while (true) { - Thread.sleep(200); - ApplicationReport report = - yarnClient.getApplicationReport(app.getApplicationId()); - if (terminatedStates.contains(report.getYarnApplicationState())) { - log.info("Application " + appName + " is stopped."); - break; - } - // kill after 10 seconds. - if ((System.currentTimeMillis() - startTime) > 10000) { - log.info("Stop operation timeout stopping, forcefully kill the app " - + appName); - yarnClient - .killApplication(app.getApplicationId(), freezeArgs.message); - break; - } - if (++pollCount % 10 == 0) { - log.info("Waiting for application " + appName + " to be stopped."); - } - } - } catch (IOException | YarnException | InterruptedException e) { - log.info("Failed to stop " + appName - + " gracefully, forcefully kill the app."); - yarnClient.killApplication(app.getApplicationId(), freezeArgs.message); - } - return EXIT_SUCCESS; - } - - @Override - public int actionStart(String appName, ActionThawArgs thaw) - throws YarnException, IOException { - validateClusterName(appName); - Path appDir = checkAppExistOnHdfs(appName); - Application application = ServiceApiUtil.loadApplication(sliderFileSystem, - appName); - ServiceApiUtil.validateAndResolveApplication(application, - sliderFileSystem, getConfig()); - // see if it is actually running and bail out; - verifyNoLiveApp(appName, "Thaw"); - ApplicationId appId = submitApp(application); - application.setId(appId.toString()); - // write app definition on to hdfs - persistApp(appDir, application); - return 0; - } - - public Map flex(String appName, Map - componentCounts) throws YarnException, IOException { - validateClusterName(appName); - Application persistedApp = ServiceApiUtil.loadApplication(sliderFileSystem, - appName); - Map original = new HashMap<>(componentCounts.size()); - for (Component persistedComp : persistedApp.getComponents()) { - String name = persistedComp.getName(); - if (componentCounts.containsKey(persistedComp.getName())) { - original.put(name, persistedComp.getNumberOfContainers()); - persistedComp.setNumberOfContainers(componentCounts.get(name)); - } - } - if (original.size() < componentCounts.size()) { - componentCounts.keySet().removeAll(original.keySet()); - throw new YarnException("Components " + componentCounts.keySet() - + " do not exist in app definition."); - } - jsonSerDeser - .save(sliderFileSystem.getFileSystem(), ServiceApiUtil.getAppJsonPath( - sliderFileSystem, appName), persistedApp, true); - log.info("Updated app definition file for components " + componentCounts - .keySet()); - - ApplicationReport instance = findInstance(appName); - if (instance != null) { - log.info("Flexing running app " + appName); - SliderClusterProtocol appMaster = connect(instance); - SliderClusterOperations clusterOps = - new SliderClusterOperations(appMaster); - clusterOps.flex(componentCounts); - for (Entry componentCount : componentCounts.entrySet()) { - log.info( - "Application name = " + appName + ", Component name = " + - componentCount.getKey() + ", number of containers updated " + - "from " + original.get(componentCount.getKey()) + " to " + - componentCount.getValue()); - } - } else { - String message = "Application " + appName + "does not exist in RM. "; - throw new YarnException(message); - } - return original; - } - - /** - * Connect to a live cluster and get its current state - * - * @param appName the cluster name - * @return its description - */ - @VisibleForTesting - public Application getApplication(String appName) - throws YarnException, IOException { - validateClusterName(appName); - SliderClusterOperations clusterOperations = - createClusterOperations(appName); - return clusterOperations.getApplication(); - } - - private ClientAMProtocol connectToAM(String appName) - throws IOException, YarnException { - if (applicationId == null) { - Application persistedApp = ServiceApiUtil.loadApplication(sliderFileSystem, - appName); - if (persistedApp == null) { - throw new YarnException("Application " + appName - + " doesn't exist on hdfs. Please check if the app exists in RM"); - } - applicationId = ApplicationId.fromString(persistedApp.getId()); - } - // Wait until app becomes running. - long startTime = System.currentTimeMillis(); - int pollCount = 0; - ApplicationReport appReport = null; - while (true) { - appReport = yarnClient.getApplicationReport(applicationId); - YarnApplicationState state = appReport.getYarnApplicationState(); - if (state == RUNNING) { - break; - } - if (terminatedStates.contains(state)) { - throw new YarnException( - "Failed to getStatus " + applicationId + ": " + appReport - .getDiagnostics()); - } - long elapsedMillis = System.currentTimeMillis() - startTime; - // if over 5 min, quit - if (elapsedMillis >= 300000) { - throw new YarnException( - "Timed out while waiting for application " + applicationId - + " to be running"); - } - - if (++pollCount % 10 == 0) { - log.info("Waiting for application {} to be running, current state is {}", - applicationId, state); - } - try { - Thread.sleep(3000); - } catch (InterruptedException ie) { - String msg = - "Interrupted while waiting for application " + applicationId - + " to be running."; - throw new YarnException(msg, ie); - } - } - - // Make the connection - InetSocketAddress address = NetUtils - .createSocketAddrForHost(appReport.getHost(), appReport.getRpcPort()); - return ClientAMProxy - .createProxy(getConfig(), ClientAMProtocol.class, - UserGroupInformation.getCurrentUser(), rpc, address); - } - - public Application getStatus(String appName) - throws IOException, YarnException { - ClientAMProtocol proxy = connectToAM(appName); - GetStatusResponseProto response = - proxy.getStatus(GetStatusRequestProto.newBuilder().build()); - Application app = jsonSerDeser.fromJson(response.getStatus()); - return app; - } - - - /** - * Bond to a running cluster - * @param clustername cluster name - * @return the AM RPC client - * @throws SliderException if the cluster is unkown - */ - private SliderClusterProtocol bondToCluster(String clustername) throws - YarnException, - IOException { - if (clustername == null) { - throw unknownClusterException("(undefined)"); - } - ApplicationReport instance = findInstance(clustername, - SliderUtils.getAllLiveAppStates()); - if (null == instance) { - throw unknownClusterException(clustername); - } - return connect(instance); - } - - /** - * Create a cluster operations instance against a given cluster - * @param clustername cluster name - * @return a bonded cluster operations instance - * @throws YarnException YARN issues - * @throws IOException IO problems - */ - private SliderClusterOperations createClusterOperations(String clustername) throws - YarnException, - IOException { - SliderClusterProtocol sliderAM = bondToCluster(clustername); - return new SliderClusterOperations(sliderAM); - } - - /** - * Generate an exception for an unknown cluster - * @param clustername cluster name - * @return an exception with text and a relevant exit code - */ - public UnknownApplicationInstanceException unknownClusterException(String clustername) { - return UnknownApplicationInstanceException.unknownInstance(clustername); - } - - @Override - public String toString() { - return "Slider Client in state " + getServiceState() - + " and Slider Application Instance " + deployedClusterName; - } - - /** - * Get all YARN applications - * @return a possibly empty list - * @throws YarnException - * @throws IOException - */ - @VisibleForTesting - public List getApplications() - throws YarnException, IOException { - return yarnClient.getApplications(); - } - - @Override - public int actionResolve(ActionResolveArgs args) - throws YarnException, IOException { - // as this is an API entry point, validate - // the arguments - args.validate(); - String path = SliderRegistryUtils.resolvePath(args.path); - ServiceRecordMarshal serviceRecordMarshal = new ServiceRecordMarshal(); - try { - if (args.list) { - File destDir = args.destdir; - if (destDir != null && !destDir.exists() && !destDir.mkdirs()) { - throw new IOException("Failed to create directory: " + destDir); - } - - - Map recordMap; - Map znodes; - try { - znodes = statChildren(registryOperations, path); - recordMap = extractServiceRecords(registryOperations, - path, - znodes.values()); - } catch (PathNotFoundException e) { - // treat the root directory as if if is always there - - if ("/".equals(path)) { - znodes = new HashMap<>(0); - recordMap = new HashMap<>(0); - } else { - throw e; - } - } - // subtract all records from the znodes map to get pure directories - log.info("Entries: {}", znodes.size()); - - for (String name : znodes.keySet()) { - println(" " + name); - } - println(""); - - log.info("Service records: {}", recordMap.size()); - for (Entry recordEntry : recordMap.entrySet()) { - String name = recordEntry.getKey(); - ServiceRecord instance = recordEntry.getValue(); - String json = serviceRecordMarshal.toJson(instance); - if (destDir == null) { - println(name); - println(json); - } else { - String filename = RegistryPathUtils.lastPathEntry(name) + ".json"; - File jsonFile = new File(destDir, filename); - write(jsonFile, serviceRecordMarshal.toBytes(instance)); - } - } - } else { - // resolve single entry - ServiceRecord instance = resolve(path); - File outFile = args.out; - if (args.destdir != null) { - outFile = new File(args.destdir, RegistryPathUtils.lastPathEntry(path)); - } - if (outFile != null) { - write(outFile, serviceRecordMarshal.toBytes(instance)); - } else { - println(serviceRecordMarshal.toJson(instance)); - } - } - } catch (PathNotFoundException | NoRecordException e) { - // no record at this path - throw new NotFoundException(e, path); - } - return EXIT_SUCCESS; - } - - @Override - public int actionRegistry(ActionRegistryArgs registryArgs) throws - YarnException, - IOException { - // as this is also a test entry point, validate - // the arguments - registryArgs.validate(); - try { - if (registryArgs.list) { - actionRegistryList(registryArgs); - } else if (registryArgs.listConf) { - // list the configurations - actionRegistryListConfigsYarn(registryArgs); - } else if (registryArgs.listExports) { - // list the exports - actionRegistryListExports(registryArgs); - } else if (isSet(registryArgs.getConf)) { - // get a configuration - PublishedConfiguration publishedConfiguration = - actionRegistryGetConfig(registryArgs); - outputConfig(publishedConfiguration, registryArgs); - } else if (isSet(registryArgs.getExport)) { - // get a export group - PublishedExports publishedExports = - actionRegistryGetExport(registryArgs); - outputExport(publishedExports, registryArgs); - } else { - // it's an unknown command - log.info(CommonArgs.usage(serviceArgs, ACTION_DIAGNOSTICS)); - return EXIT_USAGE; - } -// JDK7 - } catch (FileNotFoundException e) { - log.info("{}", e.toString()); - log.debug("{}", e, e); - return EXIT_NOT_FOUND; - } catch (PathNotFoundException e) { - log.info("{}", e.toString()); - log.debug("{}", e, e); - return EXIT_NOT_FOUND; - } - return EXIT_SUCCESS; - } - - /** - * Registry operation - * - * @param registryArgs registry Arguments - * @return the instances (for tests) - * @throws YarnException YARN problems - * @throws IOException Network or other problems - */ - @VisibleForTesting - public Collection actionRegistryList( - ActionRegistryArgs registryArgs) - throws YarnException, IOException { - String serviceType = registryArgs.serviceType; - String name = registryArgs.name; - RegistryOperations operations = getRegistryOperations(); - Collection serviceRecords; - if (StringUtils.isEmpty(name)) { - String path = serviceclassPath(currentUser(), serviceType); - - try { - Map recordMap = - listServiceRecords(operations, path); - if (recordMap.isEmpty()) { - throw new UnknownApplicationInstanceException( - "No applications registered under " + path); - } - serviceRecords = recordMap.values(); - } catch (PathNotFoundException e) { - throw new NotFoundException(path, e); - } - } else { - ServiceRecord instance = lookupServiceRecord(registryArgs); - serviceRecords = new ArrayList<>(1); - serviceRecords.add(instance); - } - - for (ServiceRecord serviceRecord : serviceRecords) { - logInstance(serviceRecord, registryArgs.verbose); - } - return serviceRecords; - } - - @Override - public int actionDiagnostic(ActionDiagnosticArgs diagnosticArgs) { - try { - if (diagnosticArgs.client) { - actionDiagnosticClient(diagnosticArgs); - } else if (diagnosticArgs.application) { - // TODO print configs of application - get from AM - } else if (diagnosticArgs.yarn) { - // This method prints yarn nodes info and yarn configs. - // We can just use yarn node CLI instead which is much more richful - // for yarn configs, this method reads local config which is only client - // config not cluster configs. -// actionDiagnosticYarn(diagnosticArgs); - } else if (diagnosticArgs.credentials) { - // actionDiagnosticCredentials internall only runs a bare 'klist' command... - // IMHO, the user can just run klist on their own with extra options supported, don't - // actually see the point of this method. -// actionDiagnosticCredentials(); - } else if (diagnosticArgs.all) { - actionDiagnosticAll(diagnosticArgs); - } else if (diagnosticArgs.level) { - // agent is removed - } else { - // it's an unknown option - log.info(CommonArgs.usage(serviceArgs, ACTION_DIAGNOSTICS)); - return EXIT_USAGE; - } - } catch (Exception e) { - log.error(e.toString()); - return EXIT_FALSE; - } - return EXIT_SUCCESS; - } - - private void actionDiagnosticAll(ActionDiagnosticArgs diagnosticArgs) - throws IOException, YarnException { - // assign application name from param to each sub diagnostic function - actionDiagnosticClient(diagnosticArgs); - // actionDiagnosticSlider only prints the agent location on hdfs, - // which is invalid now. - // actionDiagnosticCredentials only runs 'klist' command, IMHO, the user - // can just run klist on its own with extra options supported, don't - // actually see the point of this method. - } - - private void actionDiagnosticClient(ActionDiagnosticArgs diagnosticArgs) - throws SliderException, IOException { - try { - String currentCommandPath = getCurrentCommandPath(); - SliderVersionInfo.loadAndPrintVersionInfo(log); - String clientConfigPath = getClientConfigPath(); - String jdkInfo = getJDKInfo(); - println("The slider command path: %s", currentCommandPath); - println("The slider-client.xml used by current running command path: %s", - clientConfigPath); - println(jdkInfo); - - // security info - Configuration config = getConfig(); - if (isHadoopClusterSecure(config)) { - println("Hadoop Cluster is secure"); - println("Login user is %s", UserGroupInformation.getLoginUser()); - println("Current user is %s", UserGroupInformation.getCurrentUser()); - - } else { - println("Hadoop Cluster is insecure"); - } - - // verbose? - if (diagnosticArgs.verbose) { - // do the environment - Map env = getSystemEnv(); - Set envList = ConfigHelper.sortedConfigKeys(env.entrySet()); - StringBuilder builder = new StringBuilder("Environment variables:\n"); - for (String key : envList) { - builder.append(key).append("=").append(env.get(key)).append("\n"); - } - println(builder.toString()); - - // Java properties - builder = new StringBuilder("JVM Properties\n"); - Map props = - sortedMap(toMap(System.getProperties())); - for (Entry entry : props.entrySet()) { - builder.append(entry.getKey()).append("=") - .append(entry.getValue()).append("\n"); - } - - println(builder.toString()); - - // then the config - println("Slider client configuration:\n" + ConfigHelper.dumpConfigToString(config)); - } - - validateSliderClientEnvironment(log); - } catch (SliderException | IOException e) { - log.error(e.toString()); - throw e; - } - - } - - /** - * Kerberos Diagnostics - * @param args CLI arguments - * @return exit code - * @throws SliderException - * @throws IOException - */ - @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") - private int actionKDiag(ActionKDiagArgs args) - throws Exception { - PrintStream out; - boolean closeStream = false; - if (args.out != null) { - out = new PrintStream(args.out, "UTF-8"); - closeStream = true; - } else { - out = System.err; - } - try { - KerberosDiags kdiags = new KerberosDiags(getConfig(), - out, - args.services, - args.keytab, - args.principal, - args.keylen, - args.secure); - kdiags.execute(); - } catch (KerberosDiags.KerberosDiagsFailure e) { - log.error(e.toString()); - log.debug(e.toString(), e); - throw e; - } catch (Exception e) { - log.error("Kerberos Diagnostics", e); - throw e; - } finally { - if (closeStream) { - out.flush(); - out.close(); - } - } - return 0; - } - - /** - * Log a service record instance - * @param instance record - * @param verbose verbose logging of all external endpoints - */ - private void logInstance(ServiceRecord instance, - boolean verbose) { - if (!verbose) { - log.info("{}", instance.get(YarnRegistryAttributes.YARN_ID, "")); - } else { - log.info("{}: ", instance.get(YarnRegistryAttributes.YARN_ID, "")); - logEndpoints(instance); - } - } - - /** - * Log the external endpoints of a service record - * @param instance service record instance - */ - private void logEndpoints(ServiceRecord instance) { - List endpoints = instance.external; - for (Endpoint endpoint : endpoints) { - log.info(endpoint.toString()); - } - } - - /** - * list configs available for an instance - * - * @param registryArgs registry Arguments - * @throws YarnException YARN problems - * @throws IOException Network or other problems - */ - public void actionRegistryListConfigsYarn(ActionRegistryArgs registryArgs) - throws YarnException, IOException { - - ServiceRecord instance = lookupServiceRecord(registryArgs); - - RegistryRetriever retriever = new RegistryRetriever(getConfig(), instance); - PublishedConfigSet configurations = - retriever.getConfigurations(!registryArgs.internal); - PrintStream out = null; - try { - if (registryArgs.out != null) { - out = new PrintStream(registryArgs.out, "UTF-8"); - } else { - out = System.out; - } - for (String configName : configurations.keys()) { - if (!registryArgs.verbose) { - out.println(configName); - } else { - PublishedConfiguration published = configurations.get(configName); - out.printf("%s: %s%n", configName, published.description); - } - } - } finally { - if (registryArgs.out != null && out != null) { - out.flush(); - out.close(); - } - } - } - - /** - * list exports available for an instance - * - * @param registryArgs registry Arguments - * @throws YarnException YARN problems - * @throws IOException Network or other problems - */ - public void actionRegistryListExports(ActionRegistryArgs registryArgs) - throws YarnException, IOException { - ServiceRecord instance = lookupServiceRecord(registryArgs); - - RegistryRetriever retriever = new RegistryRetriever(getConfig(), instance); - PublishedExportsSet exports = - retriever.getExports(!registryArgs.internal); - PrintStream out = null; - boolean streaming = false; - try { - if (registryArgs.out != null) { - out = new PrintStream(registryArgs.out, "UTF-8"); - streaming = true; - log.debug("Saving output to {}", registryArgs.out); - } else { - out = System.out; - } - log.debug("Number of exports: {}", exports.keys().size()); - for (String exportName : exports.keys()) { - if (streaming) { - log.debug(exportName); - } - if (!registryArgs.verbose) { - out.println(exportName); - } else { - PublishedExports published = exports.get(exportName); - out.printf("%s: %s%n", exportName, published.description); - } - } - } finally { - if (streaming) { - out.flush(); - out.close(); - } - } - } - - /** - * list configs available for an instance - * - * @param registryArgs registry Arguments - * @throws YarnException YARN problems - * @throws IOException Network or other problems - * @throws FileNotFoundException if the config is not found - */ - @VisibleForTesting - public PublishedConfiguration actionRegistryGetConfig(ActionRegistryArgs registryArgs) - throws YarnException, IOException { - return ClientUtils.getConfigFromRegistry(getRegistryOperations(), - getConfig(), registryArgs.getConf, registryArgs.name, registryArgs.user, - !registryArgs.internal); - } - - /** - * get a specific export group - * - * @param registryArgs registry Arguments - * - * @throws YarnException YARN problems - * @throws IOException Network or other problems - * @throws FileNotFoundException if the config is not found - */ - @VisibleForTesting - public PublishedExports actionRegistryGetExport(ActionRegistryArgs registryArgs) - throws YarnException, IOException { - ServiceRecord instance = lookupServiceRecord(registryArgs); - - RegistryRetriever retriever = new RegistryRetriever(getConfig(), instance); - boolean external = !registryArgs.internal; - PublishedExportsSet exports = retriever.getExports(external); - - PublishedExports published = retriever.retrieveExports(exports, - registryArgs.getExport, - external); - return published; - } - - /** - * write out the config. If a destination is provided and that dir is a - * directory, the entry is written to it with the name provided + extension, - * else it is printed to standard out. - * @param published published config - * @param registryArgs registry Arguments - * @throws BadCommandArgumentsException - * @throws IOException - */ - private void outputConfig(PublishedConfiguration published, - ActionRegistryArgs registryArgs) throws - BadCommandArgumentsException, - IOException { - // decide whether or not to print - String entry = registryArgs.getConf; - String format = registryArgs.format; - String output = ClientUtils.saveOrReturnConfig(published, - registryArgs.format, registryArgs.out, entry + "." + format); - if (output != null) { - print(output); - } - } - - /** - * write out the config - * @param published - * @param registryArgs - * @throws BadCommandArgumentsException - * @throws IOException - */ - private void outputExport(PublishedExports published, - ActionRegistryArgs registryArgs) throws - BadCommandArgumentsException, - IOException { - // decide whether or not to print - String entry = registryArgs.getExport; - String format = ConfigFormat.JSON.toString(); - ConfigFormat configFormat = ConfigFormat.resolve(format); - if (configFormat == null || configFormat != ConfigFormat.JSON) { - throw new BadCommandArgumentsException( - "Unknown/Unsupported format %s . Only JSON is supported.", format); - } - - PublishedExportsOutputter outputter = - PublishedExportsOutputter.createOutputter(configFormat, - published); - boolean print = registryArgs.out == null; - if (!print) { - File destFile; - destFile = registryArgs.out; - if (destFile.isDirectory()) { - // creating it under a directory - destFile = new File(destFile, entry + "." + format); - } - log.info("Destination path: {}", destFile); - outputter.save(destFile); - } else { - print(outputter.asString()); - } - } - - /** - * Look up an instance - * @return instance data - * @throws SliderException other failures - * @throws IOException IO problems or wrapped exceptions - */ - private ServiceRecord lookupServiceRecord(ActionRegistryArgs registryArgs) throws - SliderException, - IOException { - return ClientUtils.lookupServiceRecord(getRegistryOperations(), - registryArgs.user, registryArgs.serviceType, registryArgs.name); - } - - /** - * - * Look up an instance - * @param path path - * @return instance data - * @throws NotFoundException no path/no service record - * at the end of the path - * @throws SliderException other failures - * @throws IOException IO problems or wrapped exceptions - */ - public ServiceRecord resolve(String path) - throws IOException, SliderException { - return ClientUtils.resolve(getRegistryOperations(), path); - } - - /** - * List instances in the registry for the current user - * @return a list of slider registry instances - * @throws IOException Any IO problem ... including no path in the registry - * to slider service classes for this user - * @throws SliderException other failures - */ - - public Map listRegistryInstances() - throws IOException, SliderException { - Map recordMap = listServiceRecords( - getRegistryOperations(), - serviceclassPath(currentUser(), SliderKeys.APP_TYPE)); - return recordMap; - } - - /** - * List instances in the registry - * @return the instance IDs - * @throws IOException - * @throws YarnException - */ - public List listRegisteredSliderInstances() throws - IOException, - YarnException { - try { - Map recordMap = listServiceRecords( - getRegistryOperations(), - serviceclassPath(currentUser(), SliderKeys.APP_TYPE)); - return new ArrayList<>(recordMap.keySet()); - } catch (PathNotFoundException e) { - log.debug("No registry path for slider instances for current user: {}", e, e); - // no entries: return an empty list - return new ArrayList<>(0); - } catch (IOException | YarnException e) { - throw e; - } catch (Exception e) { - throw new IOException(e); - } - } - - /** - * Start the registry if it is not there yet - * @return the registry service - * @throws SliderException - * @throws IOException - */ - private synchronized RegistryOperations maybeStartYarnRegistry() - throws SliderException, IOException { - - if (registryOperations == null) { - registryOperations = startRegistryOperationsService(); - } - return registryOperations; - } - - @Override - public RegistryOperations getRegistryOperations() - throws SliderException, IOException { - return maybeStartYarnRegistry(); - } - - /** - * Output to standard out/stderr (implementation specific detail) - * @param src source - */ - private static void print(CharSequence src) { - clientOutputStream.print(src); - } - - /** - * Output to standard out/stderr with a newline after - * @param message message - */ - private static void println(String message) { - clientOutputStream.println(message); - } - /** - * Output to standard out/stderr with a newline after, formatted - * @param message message - * @param args arguments for string formatting - */ - private static void println(String message, Object ... args) { - clientOutputStream.println(String.format(message, args)); - } - - /** - * Implement the lookup action. - * @param args Action arguments - * @return 0 if the entry was found - * @throws IOException - * @throws YarnException - * @throws UnknownApplicationInstanceException if a specific instance - * was named but it was not found - */ - @VisibleForTesting - public int actionLookup(ActionLookupArgs args) - throws IOException, YarnException { - try { - ApplicationId id = ConverterUtils.toApplicationId(args.id); - ApplicationReport report = yarnClient.getApplicationReport(id); - SerializedApplicationReport sar = new SerializedApplicationReport(report); - ApplicationReportSerDeser serDeser = new ApplicationReportSerDeser(); - if (args.outputFile != null) { - serDeser.save(sar, args.outputFile); - } else { - println(serDeser.toJson(sar)); - } - } catch (IllegalArgumentException e) { - throw new BadCommandArgumentsException(e, "%s : %s", args, e); - } catch (ApplicationAttemptNotFoundException | ApplicationNotFoundException notFound) { - throw new NotFoundException(notFound, notFound.toString()); - } - return EXIT_SUCCESS; - } - - @Override - public int actionDependency(ActionDependencyArgs args) throws IOException, - YarnException { - String currentUser = getUsername(); - log.info("Running command as user {}", currentUser); - - String version = getSliderVersion(); - Path dependencyLibTarGzip = sliderFileSystem.getDependencyTarGzip(); - - // Check if dependency has already been uploaded, in which case log - // appropriately and exit success (unless overwrite has been requested) - if (sliderFileSystem.isFile(dependencyLibTarGzip) && !args.overwrite) { - 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( - SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_NAME + "_", - SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_EXT); - // copy all jars - tarGzipFolder(libDirs, tempLibTarGzipFile, createJarFilter()); - - log.info("Uploading dependency for AM (version {}) from {} to {}", - version, tempLibTarGzipFile.toURI(), dependencyLibTarGzip.toUri()); - sliderFileSystem.copyLocalFileToHdfs(tempLibTarGzipFile, - dependencyLibTarGzip, new FsPermission( - SliderKeys.SLIDER_DEPENDENCY_DIR_PERMISSIONS)); - return EXIT_SUCCESS; - } else { - return EXIT_FALSE; - } - } - - private int actionHelp(String actionName) throws YarnException, IOException { - throw new UsageException(CommonArgs.usage(serviceArgs, actionName)); - } - - /** - * List the nodes in the cluster, possibly filtering by node state or label. - * - * @param args argument list - * @return a possibly empty list of nodes in the cluster - * @throws IOException IO problems - * @throws YarnException YARN problems - */ - @Override - public NodeInformationList listYarnClusterNodes(ActionNodesArgs args) - throws YarnException, IOException { - return yarnClient.listNodes(args.label, args.healthy); - } - - /** - * List the nodes in the cluster, possibly filtering by node state or label. - * - * @param args argument list - * @return a possibly empty list of nodes in the cluster - * @throws IOException IO problems - * @throws YarnException YARN problems - */ - public NodeInformationList listInstanceNodes(String instance, ActionNodesArgs args) - throws YarnException, IOException { - // TODO - log.info("listInstanceNodes {}", instance); - SliderClusterOperations clusterOps = - new SliderClusterOperations(bondToCluster(instance)); - return clusterOps.getLiveNodes(); - } - - /** - * List the nodes in the cluster, possibly filtering by node state or label. - * Prints them to stdout unless the args names a file instead. - * @param args argument list - * @throws IOException IO problems - * @throws YarnException YARN problems - */ - public int actionNodes(String instance, ActionNodesArgs args) throws YarnException, IOException { - - args.instance = instance; - NodeInformationList nodes; - if (SliderUtils.isUnset(instance)) { - nodes = listYarnClusterNodes(args); - } else { - nodes = listInstanceNodes(instance, args); - } - log.debug("Node listing for {} has {} nodes", args, nodes.size()); - JsonSerDeser serDeser = NodeInformationList.createSerializer(); - if (args.outputFile != null) { - serDeser.save(nodes, args.outputFile); - } else { - println(serDeser.toJson(nodes)); - } - return 0; - } - - /** - * Save/list tokens. This is for testing oozie integration - * @param args commands - * @return status - */ - private int actionTokens(ActionTokensArgs args) - throws IOException, YarnException { - return new TokensOperation().actionTokens(args, - sliderFileSystem.getFileSystem(), - getConfig(), - yarnClient); - } - - @VisibleForTesting - public ApplicationReport monitorAppToRunning(Duration duration) - throws YarnException, IOException { - return yarnClient.monitorAppToState(applicationId, RUNNING, duration); - } -} - - diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java deleted file mode 100644 index f1bf2ad..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java +++ /dev/null @@ -1,258 +0,0 @@ -/* - * 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.slider.client; - -import org.apache.hadoop.registry.client.api.RegistryOperations; -import org.apache.hadoop.service.Service; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.types.NodeInformationList; -import org.apache.slider.common.params.AbstractClusterBuildingActionArgs; -import org.apache.slider.common.params.ActionAMSuicideArgs; -import org.apache.slider.common.params.ActionClientArgs; -import org.apache.hadoop.yarn.service.client.params.ActionDependencyArgs; -import org.apache.slider.common.params.ActionDiagnosticArgs; -import org.apache.hadoop.yarn.service.client.params.ActionFlexArgs; -import org.apache.slider.common.params.ActionFreezeArgs; -import org.apache.slider.common.params.ActionKeytabArgs; -import org.apache.slider.common.params.ActionNodesArgs; -import org.apache.slider.common.params.ActionKillContainerArgs; -import org.apache.slider.common.params.ActionListArgs; -import org.apache.slider.common.params.ActionRegistryArgs; -import org.apache.slider.common.params.ActionResolveArgs; -import org.apache.slider.common.params.ActionResourceArgs; -import org.apache.slider.common.params.ActionStatusArgs; -import org.apache.slider.common.params.ActionThawArgs; -import org.apache.slider.common.params.ActionUpgradeArgs; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.SliderException; - -import java.io.IOException; - -/** - * Interface of those method calls in the slider API that are intended - * for direct public invocation. - *

- * Stability: evolving - */ -public interface SliderClientAPI extends Service { - - int actionDestroy(String clustername) throws YarnException, IOException; - - /** - * AM to commit an asynchronous suicide - */ - int actionAmSuicide(String clustername, - ActionAMSuicideArgs args) throws YarnException, IOException; - - /** - * Manage keytabs leveraged by slider - * - * @param keytabInfo the arguments needed to manage the keytab - * @throws YarnException Yarn problems - * @throws IOException other problems - * @throws BadCommandArgumentsException bad arguments. - */ - int actionKeytab(ActionKeytabArgs keytabInfo) - throws YarnException, IOException; - - /** - * Manage file resources leveraged by slider - * - * @param resourceInfo the arguments needed to manage the resource - * @throws YarnException Yarn problems - * @throws IOException other problems - * @throws BadCommandArgumentsException bad arguments. - */ - int actionResource(ActionResourceArgs resourceInfo) - throws YarnException, IOException; - - /** - * Perform client operations such as install or configure - * - * @param clientInfo the arguments needed for client operations - * - * @throws SliderException bad arguments. - * @throws IOException problems related to package and destination folders - */ - int actionClient(ActionClientArgs clientInfo) - throws IOException, YarnException; - - /** - * Update the cluster specification - * - * @param clustername cluster name - * @param buildInfo the arguments needed to update the cluster - * @throws YarnException Yarn problems - * @throws IOException other problems - */ - int actionUpdate(String clustername, - AbstractClusterBuildingActionArgs buildInfo) - throws YarnException, IOException; - - /** - * Upgrade the cluster with a newer version of the application - * - * @param clustername cluster name - * @param buildInfo the arguments needed to upgrade the cluster - * @throws YarnException Yarn problems - * @throws IOException other problems - */ - int actionUpgrade(String clustername, - ActionUpgradeArgs buildInfo) - throws YarnException, IOException; - - /** - * Implement the list action: list all nodes - * @return exit code of 0 if a list was created - */ - int actionList(String clustername, ActionListArgs args) throws IOException, YarnException; - - - int actionFlex(String name, ActionFlexArgs args) throws YarnException, - IOException; - - /** - * Test for a cluster existing probe for a cluster of the given name existing - * in the filesystem. If the live param is set, it must be a live cluster - * @return exit code - */ - int actionExists(String name, boolean checkLive) throws YarnException, IOException; - - /** - * Kill a specific container of the cluster - * @param name cluster name - * @param args arguments - * @return exit code - * @throws YarnException - * @throws IOException - */ - int actionKillContainer(String name, ActionKillContainerArgs args) - throws YarnException, IOException; - - /** - * Status operation - * - * @param clustername cluster name - * @param statusArgs status arguments - * @return 0 -for success, else an exception is thrown - * @throws YarnException - * @throws IOException - */ - int actionStatus(String clustername, ActionStatusArgs statusArgs) - throws YarnException, IOException; - - /** - * Status operation which returns the status object as a string instead of - * printing it to the console or file. - * - * @param clustername cluster name - * @return cluster status details - * @throws YarnException - * @throws IOException - */ - Application actionStatus(String clustername) throws YarnException, IOException; - - /** - * Version Details - * @return exit code - */ - int actionVersion(); - - /** - * Stop the cluster - * - * @param clustername cluster name - * @param freezeArgs arguments to the stop - * @return EXIT_SUCCESS if the cluster was not running by the end of the operation - */ - int actionStop(String clustername, ActionFreezeArgs freezeArgs) - throws YarnException, IOException; - - /** - * Restore a cluster - */ - int actionStart(String clustername, ActionThawArgs thaw) throws YarnException, IOException; - - /** - * Registry operation - * - * @param args registry Arguments - * @return 0 for success, -1 for some issues that aren't errors, just failures - * to retrieve information (e.g. no configurations for that entry) - * @throws YarnException YARN problems - * @throws IOException Network or other problems - */ - int actionResolve(ActionResolveArgs args) - throws YarnException, IOException; - - /** - * Registry operation - * - * @param registryArgs registry Arguments - * @return 0 for success, -1 for some issues that aren't errors, just failures - * to retrieve information (e.g. no configurations for that entry) - * @throws YarnException YARN problems - * @throws IOException Network or other problems - */ - int actionRegistry(ActionRegistryArgs registryArgs) - throws YarnException, IOException; - - /** - * diagnostic operation - * - * @param diagnosticArgs diagnostic Arguments - * @return 0 for success, -1 for some issues that aren't errors, just - * failures to retrieve information (e.g. no application name - * specified) - * @throws YarnException YARN problems - * @throws IOException Network or other problems - */ - int actionDiagnostic(ActionDiagnosticArgs diagnosticArgs); - - /** - * Get the registry binding. As this may start the registry, it can take time - * and fail - * @return the registry - */ - RegistryOperations getRegistryOperations() - throws SliderException, IOException; - - /** - * Upload all Slider AM and agent dependency libraries to HDFS, so that they - * do not need to be uploaded with every create call. This operation is - * Slider version specific. So it needs to be invoked for every single - * version of slider/slider-client. - * - * @throws SliderException - * @throws IOException - */ - int actionDependency(ActionDependencyArgs dependencyArgs) throws IOException, - YarnException; - - /** - * List the nodes - * @param args - * @return - * @throws YarnException - * @throws IOException - */ - NodeInformationList listYarnClusterNodes(ActionNodesArgs args) - throws YarnException, IOException; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java deleted file mode 100644 index e1b578c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java +++ /dev/null @@ -1,294 +0,0 @@ -/* - * 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.slider.client; - -import com.google.common.base.Preconditions; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.yarn.api.ApplicationClientProtocol; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.NodeState; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.slider.api.types.NodeInformation; -import org.apache.slider.api.types.NodeInformationList; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.slider.common.tools.Duration; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.EnumSet; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -//TODO, Remove this class and YarnAppListClient -// why do we need so many yarn client wrappers ? -// - yarn client already provides most of functionality already - -/** - * A class that extends visibility to some of the YarnClientImpl - * members and data structures, and factors out pure-YARN operations - * from the slider entry point service - */ -public class SliderYarnClientImpl extends YarnClientImpl { - protected static final Logger log = LoggerFactory.getLogger(SliderYarnClientImpl.class); - - /** - * Get the RM Client RPC interface - * @return an RPC interface valid after initialization and authentication - */ - public ApplicationClientProtocol getRmClient() { - return rmClient; - } - - /** - * List Slider deployedinstances belonging to a specific user in a - * given set of states and filtered by an application name tag. - *

- * Deployed means: known about in the YARN cluster; it will include all apps - * in the specified set of states and tagged with the specified app name. - * - * @param user - * user: "" means all users - * @param appStates - * filter by a set of YarnApplicationState - * @param appname - * an application name tag in the format defined by - * {@link SliderUtils#createNameTag(String)} - * @return a possibly empty list of Slider AMs - * @throws YarnException - * @throws IOException - */ - public List listDeployedInstances(String user, - EnumSet appStates, String appname) - throws YarnException, IOException { - Preconditions.checkArgument(user != null, "Null User"); - Set types = new HashSet<>(1); - types.add(SliderKeys.APP_TYPE); - Set tags = null; - if (appname != null) { - tags = Collections.singleton(SliderUtils.createNameTag(appname)); - } - List allApps = getApplications(types, appStates, tags); - List results = new ArrayList<>(); - for (ApplicationReport report : allApps) { - if (StringUtils.isEmpty(user) || user.equals(report.getUser())) { - results.add(report); - } - } - return results; - } - - /** - * Helper method to determine if a cluster application is running -or - * is earlier in the lifecycle - * @param app application report - * @return true if the application is considered live - */ - public boolean isApplicationLive(ApplicationReport app) { - Preconditions.checkArgument(app != null, "Null app report"); - - return app.getYarnApplicationState().ordinal() <= YarnApplicationState.RUNNING.ordinal(); - } - - /** - * find all live instances of a specific app -if there is >1 in the cluster, - * this returns them all. State should be running or less - * @param user user - * @param appname application name - * @return the list of all matching application instances - */ - public List findAllLiveInstances(String user, - String appname) throws YarnException, IOException { - Preconditions.checkArgument(StringUtils.isNotEmpty(appname), - "Null/empty application name"); - return listDeployedInstances(user, SliderUtils.getAllLiveAppStates(), - appname); - } - - /** - * Find a cluster in the instance list; biased towards live instances - * @param instances list of instances - * @param appname application name - * @return the first found instance, else a failed/finished instance, or null - * if there are none of those - */ - public ApplicationReport findClusterInInstanceList(List instances, - String appname) { - Preconditions.checkArgument(instances != null, "Null instances list"); - Preconditions.checkArgument(StringUtils.isNotEmpty(appname), - "Null/empty application name"); - // sort by most recent - SliderUtils.sortApplicationsByMostRecent(instances); - ApplicationReport found = null; - for (ApplicationReport app : instances) { - if (app.getName().equals(appname)) { - if (isApplicationLive(app)) { - return app; - } - // set the found value if not set - found = found != null ? found : app; - } - } - return found; - } - - /** - * Find an app in the instance list in the desired state - * @param instances instance list - * @param appname application name - * @param desiredState yarn state desired - * @return the match or null for none - */ - public ApplicationReport findAppInInstanceList(List instances, - String appname, - YarnApplicationState desiredState) { - Preconditions.checkArgument(instances != null, "Null instances list"); - Preconditions.checkArgument(StringUtils.isNotEmpty(appname), - "Null/empty application name"); - Preconditions.checkArgument(desiredState != null, "Null desiredState"); - log.debug("Searching {} records for instance name {} in state '{}'", - instances.size(), appname, desiredState); - for (ApplicationReport app : instances) { - if (app.getName().equals(appname)) { - - YarnApplicationState appstate = - app.getYarnApplicationState(); - log.debug("app ID {} is in state {}", app.getApplicationId(), appstate); - if (appstate.equals(desiredState)) { - log.debug("match"); - return app; - } - } - } - // nothing found in desired state - log.debug("No match"); - return null; - } - - /** - * List the nodes in the cluster, possibly filtering by node state or label. - * - * @param label label to filter by -or "" for any - * @param live flag to request running nodes only - * @return a possibly empty list of nodes in the cluster - * @throws IOException IO problems - * @throws YarnException YARN problems - */ - public NodeInformationList listNodes(String label, boolean live) - throws IOException, YarnException { - Preconditions.checkArgument(label != null, "null label"); - NodeState[] states; - if (live) { - states = new NodeState[1]; - states[0] = NodeState.RUNNING; - } else { - states = new NodeState[0]; - } - List reports = getNodeReports(states); - NodeInformationList results = new NodeInformationList(reports.size()); - for (NodeReport report : reports) { - if (live && report.getNodeState() != NodeState.RUNNING) { - continue; - } - if (!label.isEmpty() && !report.getNodeLabels().contains(label)) { - continue; - } - // build node info from report - NodeInformation info = new NodeInformation(); - info.hostname = report.getNodeId().getHost(); - info.healthReport = report.getHealthReport(); - info.httpAddress = report.getHttpAddress(); - info.labels = SliderUtils.extractNodeLabel(report); - info.rackName = report.getRackName(); - info.state = report.getNodeState().toString(); - results.add(info); - } - return results; - } - - /** - * Monitor the submitted application for reaching the requested state. - * Will also report if the app reaches a later state (failed, killed, etc) - * Kill application if duration!= null & time expires. - * @param appId Application Id of application to be monitored - * @param duration how long to wait -must be more than 0 - * @param desiredState desired state. - * @return the application report -null on a timeout - * @throws YarnException - * @throws IOException - */ - public ApplicationReport monitorAppToState( - ApplicationId appId, YarnApplicationState desiredState, Duration duration) - throws YarnException, IOException { - - if (appId == null) { - throw new BadCommandArgumentsException("null application ID"); - } - if (duration.limit <= 0) { - throw new BadCommandArgumentsException("Invalid monitoring duration"); - } - log.debug("Waiting {} millis for app to reach state {} ", - duration.limit, - desiredState); - duration.start(); - try { - while (true) { - // Get application report for the appId we are interested in - - ApplicationReport r = getApplicationReport(appId); - - log.debug("queried status is\n{}", - new SliderUtils.OnDemandReportStringifier(r)); - - YarnApplicationState state = r.getYarnApplicationState(); - if (state.ordinal() >= desiredState.ordinal()) { - log.debug("App in desired state (or higher) :{}", state); - return r; - } - if (duration.getLimitExceeded()) { - log.debug( - "Wait limit of {} millis to get to state {}, exceeded; app " + - "status\n {}", - duration.limit, - desiredState, - new SliderUtils.OnDemandReportStringifier(r)); - return null; - } - - // sleep 1s. - try { - Thread.sleep(1000); - } catch (InterruptedException ignored) { - log.debug("Thread sleep in monitoring loop interrupted"); - } - } - } finally { - duration.close(); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/TokensOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/TokensOperation.java deleted file mode 100644 index 84c65b3..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/TokensOperation.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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.slider.client; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.slider.common.params.ActionTokensArgs; -import org.apache.slider.core.exceptions.BadClusterStateException; -import org.apache.slider.core.exceptions.NotFoundException; -import static org.apache.slider.core.launch.CredentialUtils.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; - -public class TokensOperation { - - private static final Logger log = LoggerFactory.getLogger(TokensOperation.class); - public static final String E_INSECURE - = "Cluster is not secure -tokens cannot be acquired"; - public static final String E_MISSING_SOURCE_FILE = "Missing source file: "; - public static final String E_NO_KEYTAB = "No keytab: "; - - public int actionTokens(ActionTokensArgs args, FileSystem fs, - Configuration conf, - YarnClientImpl yarnClient) - throws IOException, YarnException { - Credentials credentials; - String footnote = ""; - UserGroupInformation user = UserGroupInformation.getCurrentUser(); - boolean isSecure = UserGroupInformation.isSecurityEnabled(); - if (args.keytab != null) { - File keytab = args.keytab; - if (!keytab.isFile()) { - throw new NotFoundException(E_NO_KEYTAB + keytab.getAbsolutePath()); - } - String principal = args.principal; - log.info("Logging in as {} from keytab {}", principal, keytab); - user = UserGroupInformation.loginUserFromKeytabAndReturnUGI( - principal, keytab.getCanonicalPath()); - } - Credentials userCredentials = user.getCredentials(); - File output = args.output; - if (output != null) { - if (!isSecure) { - throw new BadClusterStateException(E_INSECURE); - } - credentials = new Credentials(userCredentials); - // filesystem - addRMRenewableFSDelegationTokens(conf, fs, credentials); - addRMDelegationToken(yarnClient, credentials); - if (maybeAddTimelineToken(conf, credentials) != null) { - log.debug("Added timeline token"); - } - saveTokens(output, credentials); - String filename = output.getCanonicalPath(); - footnote = String.format( - "%d tokens saved to %s%n" + "To use these in the environment:%n" - + "export %s=%s", credentials.numberOfTokens(), filename, - UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION, filename); - } else if (args.source != null) { - File source = args.source; - log.info("Reading credentials from file {}", source); - if (!source.isFile()) { - throw new NotFoundException( E_MISSING_SOURCE_FILE + source.getAbsolutePath()); - } - credentials = Credentials.readTokenStorageFile(args.source, conf); - } else { - StringBuffer origin = new StringBuffer(); - File file = locateEnvCredentials(System.getenv(), conf, - origin); - if (file != null) { - log.info("Credential Source {}", origin); - } else { - log.info("Credential source: logged in user"); - } - credentials = userCredentials; - } - // list the tokens - log.info("\n{}", dumpTokens(credentials, "\n")); - if (!footnote.isEmpty()) { - log.info(footnote); - } - return 0; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ipc/SliderClusterOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ipc/SliderClusterOperations.java deleted file mode 100644 index e89a660..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ipc/SliderClusterOperations.java +++ /dev/null @@ -1,355 +0,0 @@ -/* - * 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.slider.client.ipc; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.slider.api.ClusterNode; -import org.apache.slider.api.SliderClusterProtocol; -import org.apache.slider.api.StateValues; -import org.apache.slider.api.proto.Messages; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.types.ContainerInformation; -import org.apache.slider.api.types.NodeInformation; -import org.apache.slider.api.types.NodeInformationList; -import org.apache.slider.api.types.PingInformation; -import org.apache.slider.common.tools.Duration; -import org.apache.slider.core.exceptions.NoSuchNodeException; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.exceptions.WaitTimeoutException; -import org.apache.slider.core.persist.JsonSerDeser; -import org.codehaus.jackson.JsonParseException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - -import static org.apache.slider.api.types.RestTypeMarshalling.unmarshall; - -/** - * Cluster operations at a slightly higher level than the RPC code - */ -public class SliderClusterOperations { - protected static final Logger - log = LoggerFactory.getLogger(SliderClusterOperations.class); - - private final SliderClusterProtocol appMaster; - private static final JsonSerDeser jsonSerDeser = - new JsonSerDeser(Application.class); - private static final Messages.EmptyPayloadProto EMPTY; - static { - EMPTY = Messages.EmptyPayloadProto.newBuilder().build(); - } - - public SliderClusterOperations(SliderClusterProtocol appMaster) { - this.appMaster = appMaster; - } - - @Override - public String toString() { - final StringBuilder sb = - new StringBuilder("SliderClusterOperations{"); - sb.append("IPC binding=").append(appMaster); - sb.append('}'); - return sb.toString(); - } - - /** - * Get a node from the AM - * @param uuid uuid of node - * @return deserialized node - * @throws IOException IO problems - * @throws NoSuchNodeException if the node isn't found - */ - public ClusterNode getNode(String uuid) - throws IOException, NoSuchNodeException, YarnException { - Messages.GetNodeRequestProto req = - Messages.GetNodeRequestProto.newBuilder().setUuid(uuid).build(); - Messages.GetNodeResponseProto node = appMaster.getNode(req); - return ClusterNode.fromProtobuf(node.getClusterNode()); - } - - /** - * Unmarshall a list of nodes from a protobud response - * @param nodes node list - * @return possibly empty list of cluster nodes - * @throws IOException - */ - public List convertNodeWireToClusterNodes(List nodes) - throws IOException { - List nodeList = new ArrayList<>(nodes.size()); - for (Messages.RoleInstanceState node : nodes) { - nodeList.add(ClusterNode.fromProtobuf(node)); - } - return nodeList; - } - - /** - * Echo text (debug action) - * @param text text - * @return the text, echoed back - * @throws YarnException - * @throws IOException - */ - public String echo(String text) throws YarnException, IOException { - Messages.EchoRequestProto.Builder builder = - Messages.EchoRequestProto.newBuilder(); - builder.setText(text); - Messages.EchoRequestProto req = builder.build(); - Messages.EchoResponseProto response = appMaster.echo(req); - return response.getText(); - } - - - /** - * Connect to a live cluster and get its current state - * @return its description - */ - public Application getApplication() throws YarnException, IOException { - Messages.GetJSONClusterStatusRequestProto req = - Messages.GetJSONClusterStatusRequestProto.newBuilder().build(); - Messages.GetJSONClusterStatusResponseProto resp = - appMaster.getJSONClusterStatus(req); - String statusJson = resp.getClusterSpec(); - try { - return jsonSerDeser.fromJson(statusJson); - } catch (JsonParseException e) { - log.error("Error when parsing app json file", e); - throw e; - } - } - - /** - * Kill a container - * @param id container ID - * @return a success flag - * @throws YarnException - * @throws IOException - */ - public boolean killContainer(String id) throws - YarnException, - IOException { - Messages.KillContainerRequestProto.Builder builder = - Messages.KillContainerRequestProto.newBuilder(); - builder.setId(id); - Messages.KillContainerRequestProto req = builder.build(); - Messages.KillContainerResponseProto response = appMaster.killContainer(req); - return response.getSuccess(); - } - - /** - * List all node UUIDs in a role - * @param role role name or "" for all - * @return an array of UUID strings - * @throws IOException - * @throws YarnException - */ - public String[] listNodeUUIDsByRole(String role) throws IOException, YarnException { - Collection uuidList = innerListNodeUUIDSByRole(role); - String[] uuids = new String[uuidList.size()]; - return uuidList.toArray(uuids); - } - - public List innerListNodeUUIDSByRole(String role) throws IOException, YarnException { - Messages.ListNodeUUIDsByRoleRequestProto req = - Messages.ListNodeUUIDsByRoleRequestProto - .newBuilder() - .setRole(role) - .build(); - Messages.ListNodeUUIDsByRoleResponseProto resp = appMaster.listNodeUUIDsByRole(req); - return resp.getUuidList(); - } - - /** - * List all nodes in a role. This is a double round trip: once to list - * the nodes in a role, another to get their details - * @param role - * @return an array of ContainerNode instances - * @throws IOException - * @throws YarnException - */ - public List listClusterNodesInRole(String role) - throws IOException, YarnException { - - Collection uuidList = innerListNodeUUIDSByRole(role); - Messages.GetClusterNodesRequestProto req = - Messages.GetClusterNodesRequestProto - .newBuilder() - .addAllUuid(uuidList) - .build(); - Messages.GetClusterNodesResponseProto resp = appMaster.getClusterNodes(req); - return convertNodeWireToClusterNodes(resp.getClusterNodeList()); - } - - /** - * Get the details on a list of uuids - * @param uuids instance IDs - * @return a possibly empty list of node details - * @throws IOException - * @throws YarnException - */ - @VisibleForTesting - public List listClusterNodes(String[] uuids) - throws IOException, YarnException { - - Messages.GetClusterNodesRequestProto req = - Messages.GetClusterNodesRequestProto - .newBuilder() - .addAllUuid(Arrays.asList(uuids)) - .build(); - Messages.GetClusterNodesResponseProto resp = appMaster.getClusterNodes(req); - return convertNodeWireToClusterNodes(resp.getClusterNodeList()); - } - - /** - * Wait for an instance of a named role to be live (or past it in the lifecycle) - * @param role role to look for - * @param timeout time to wait - * @return the state. If still in CREATED, the cluster didn't come up - * in the time period. If LIVE, all is well. If >LIVE, it has shut for a reason - * @throws IOException IO - * @throws SliderException Slider - * @throws WaitTimeoutException if the wait timed out - */ - @VisibleForTesting - public int waitForRoleInstanceLive(String role, long timeout) - throws WaitTimeoutException, IOException, YarnException { - Duration duration = new Duration(timeout); - duration.start(); - boolean live = false; - int state = StateValues.STATE_CREATED; - - log.info("Waiting {} millis for a live node in role {}", timeout, role); - try { - while (!live) { - // see if there is a node in that role yet - List uuids = innerListNodeUUIDSByRole(role); - String[] containers = uuids.toArray(new String[uuids.size()]); - int roleCount = containers.length; - ClusterNode roleInstance = null; - if (roleCount != 0) { - - // if there is, get the node - roleInstance = getNode(containers[0]); - if (roleInstance != null) { - state = roleInstance.state; - live = state >= StateValues.STATE_LIVE; - } - } - if (!live) { - if (duration.getLimitExceeded()) { - throw new WaitTimeoutException( - String.format("Timeout after %d millis" + - " waiting for a live instance of type %s; " + - "instances found %d %s", - timeout, role, roleCount, - (roleInstance != null - ? (" instance -\n" + roleInstance.toString()) - : "") - )); - } else { - try { - Thread.sleep(1000); - } catch (InterruptedException ignored) { - // ignored - } - } - } - } - } finally { - duration.close(); - } - return state; - } - - public void flex(Map componentCounts) throws IOException{ - Messages.FlexComponentsRequestProto.Builder builder = - Messages.FlexComponentsRequestProto.newBuilder(); - for (Entry componentCount : componentCounts.entrySet()) { - Messages.ComponentCountProto componentProto = - Messages.ComponentCountProto.newBuilder() - .setName(componentCount.getKey()) - .setNumberOfContainers(componentCount.getValue()).build(); - builder.addComponents(componentProto); - } - appMaster.flexComponents(builder.build()); - } - - /** - * Commit (possibly delayed) AM suicide - * - * @param signal exit code - * @param text text text to log - * @param delay delay in millis - * @throws YarnException - * @throws IOException - */ - public void amSuicide(String text, int signal, int delay) - throws IOException { - Messages.AMSuicideRequestProto.Builder builder = - Messages.AMSuicideRequestProto.newBuilder(); - if (text != null) { - builder.setText(text); - } - builder.setSignal(signal); - builder.setDelay(delay); - Messages.AMSuicideRequestProto req = builder.build(); - appMaster.amSuicide(req); - } - - public List getContainers() throws IOException { - Messages.GetLiveContainersResponseProto response = appMaster - .getLiveContainers(Messages.GetLiveContainersRequestProto.newBuilder() - .build()); - return unmarshall(response); - } - - public NodeInformationList getLiveNodes() throws IOException { - Messages.GetLiveNodesResponseProto response = - appMaster.getLiveNodes(Messages.GetLiveNodesRequestProto.newBuilder().build()); - - int records = response.getNodesCount(); - NodeInformationList nil = new NodeInformationList(records); - for (int i = 0; i < records; i++) { - nil.add(unmarshall(response.getNodes(i))); - } - return nil; - } - - public NodeInformation getLiveNode(String hostname) throws IOException { - Messages.GetLiveNodeRequestProto.Builder builder = - Messages.GetLiveNodeRequestProto.newBuilder(); - builder.setName(hostname); - return unmarshall(appMaster.getLiveNode(builder.build())); - } - - public PingInformation ping(String text) throws IOException { - return null; - } - - public void stop(String text) throws IOException { - amSuicide(text, 3, 0); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/Constants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/Constants.java deleted file mode 100644 index 0e3559a..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/Constants.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.slider.common; - -public class Constants { - public static final int CONNECT_TIMEOUT = 10000; - public static final int RPC_TIMEOUT = 15000; - - public static final String HADOOP_JAAS_DEBUG = "HADOOP_JAAS_DEBUG"; - public static final String KRB5_CCNAME = "KRB5CCNAME"; - public static final String JAVA_SECURITY_KRB5_CONF - = "java.security.krb5.conf"; - public static final String JAVA_SECURITY_KRB5_REALM - = "java.security.krb5.realm"; - public static final String SUN_SECURITY_KRB5_DEBUG - = "sun.security.krb5.debug"; - public static final String SUN_SECURITY_SPNEGO_DEBUG - = "sun.security.spnego.debug"; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderXMLConfKeysForTesting.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderXMLConfKeysForTesting.java deleted file mode 100644 index 61c828e..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderXMLConfKeysForTesting.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * 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.slider.common; - -/** - * Keys shared across tests - */ -public interface SliderXMLConfKeysForTesting { - - String KEY_TEST_THAW_WAIT_TIME = "slider.test.thaw.wait.seconds"; - - int DEFAULT_THAW_WAIT_TIME_SECONDS = 60; - - - String KEY_TEST_FREEZE_WAIT_TIME = "slider.test.freeze.wait.seconds"; - - int DEFAULT_TEST_FREEZE_WAIT_TIME_SECONDS = 60; - - String KEY_TEST_TIMEOUT = "slider.test.timeout.seconds"; - - int DEFAULT_TEST_TIMEOUT_SECONDS = 30 * 60; - - String KEY_ACCUMULO_LAUNCH_TIME = - "slider.test.accumulo.launch.wait.seconds"; - int DEFAULT_ACCUMULO_LAUNCH_TIME_SECONDS = 60 * 3; - - String KEY_ACCUMULO_GO_LIVE_TIME = - "slider.test.accumulo.live.wait.seconds"; - int DEFAULT_ACCUMULO_LIVE_TIME_SECONDS = 90; - - String KEY_TEST_AGENT_ENABLED = "slider.test.agent.enabled"; - String KEY_AGENTTESTS_QUEUE_LABELED_DEFINED = "slider.test.agent.labeled.queue.enabled"; - String KEY_AGENTTESTS_LABELS_RED_BLUE_DEFINED = "slider.test.agent.labels.defined"; - String KEY_AGENTTESTS_AM_FAILURES_ENABLED = "slider.test.agent.am.failures.enabled"; - - int DEFAULT_AGENT_LAUNCH_TIME_SECONDS = 60 * 3; - - String KEY_TEST_AGENT_HOME = "slider.test.agent.home"; - String KEY_TEST_AGENT_TAR = "slider.test.agent.tar"; - - String KEY_TEST_TEARDOWN_KILLALL = "slider.test.teardown.killall"; - boolean DEFAULT_TEARDOWN_KILLALL = true; - - - /** - * Key for amount of RAM to request - */ - String KEY_TEST_YARN_RAM_REQUEST = "slider.test.yarn.ram"; - String DEFAULT_YARN_RAM_REQUEST = "192"; - - /** - * security related keys - */ - String TEST_SECURITY_DIR = "/tmp/work/security"; - - /** - * Local path to AM keytab: {@value} - */ - String KEY_TEST_AM_KEYTAB = "slider.test.am.keytab.local"; - - /** - * Is the test cluster windows? Default is: same as the local system. - * {@value} - */ - String KEY_TEST_WINDOWS_CLUSTER = "slider.test.windows.cluster"; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionAMSuicideArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionAMSuicideArgs.java deleted file mode 100644 index 04ec9e2..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionAMSuicideArgs.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.slider.common.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_AM_SUICIDE}, - commandDescription = SliderActions.DESCRIBE_ACTION_AM_SUICIDE) -public class ActionAMSuicideArgs extends AbstractActionArgs { - - @Override - public String getActionName() { - return SliderActions.ACTION_AM_SUICIDE; - } - - @Parameter(names = {ARG_MESSAGE}, - description = "reason for the action") - public String message = ""; - - @Parameter(names = {ARG_EXITCODE}, - description = "exit code") - public int exitcode = 1; - - @Parameter(names = {ARG_WAIT}, - description = "time for AM to wait before exiting") - public int waittime = 1000; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDiagnosticArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDiagnosticArgs.java deleted file mode 100644 index cb36961..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDiagnosticArgs.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.slider.common.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_DIAGNOSTICS}, - commandDescription = SliderActions.DESCRIBE_ACTION_DIAGNOSTIC) -public class ActionDiagnosticArgs extends AbstractActionArgs { - - @Override - public String getActionName() { - return SliderActions.ACTION_DIAGNOSTICS; - } - - @Parameter(names = {ARG_NAME}, - description = "the name of the running application") - public String name; - - @Parameter(names = {ARG_CLIENT}, - description = "print configuration of the slider client") - public boolean client = false; - - @Parameter(names = {ARG_APPLICATION}, - description = "print configuration of the running application") - public boolean application; - - @Parameter(names = {ARG_VERBOSE}, - description = "print out information in details") - public boolean verbose = false; - - @Parameter(names = {ARG_YARN}, - description = "print configuration of the YARN cluster") - public boolean yarn = false; - - @Parameter(names = {ARG_CREDENTIALS}, - description = "print credentials of the current user") - public boolean credentials = false; - - @Parameter(names = {ARG_ALL}, - description = "print all of the information above") - public boolean all; - - @Parameter(names = {ARG_LEVEL}, - description = "diagnose each slider configuration one by one") - public boolean level; - - /** - * 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-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKillContainerArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKillContainerArgs.java deleted file mode 100644 index e1e94bd..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKillContainerArgs.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.slider.common.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_KILL_CONTAINER}, - commandDescription = SliderActions.DESCRIBE_ACTION_KILL_CONTAINER) - -public class ActionKillContainerArgs extends AbstractActionArgs { - @Override - public String getActionName() { - return SliderActions.ACTION_KILL_CONTAINER; - } - - @Parameter(names = {ARG_ID}, - description = "ID of the container") - public String id; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionLookupArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionLookupArgs.java deleted file mode 100644 index 0888812..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionLookupArgs.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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.slider.common.params; - -import com.beust.jcommander.Parameter; -import com.beust.jcommander.Parameters; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; -import org.apache.hadoop.yarn.service.client.params.SliderActions; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.UsageException; - -import java.io.File; - -@Parameters(commandNames = { SliderActions.ACTION_LOOKUP}, - commandDescription = SliderActions.DESCRIBE_ACTION_LOOKUP) - -public class ActionLookupArgs extends AbstractActionArgs { - @Override - public String getActionName() { - return SliderActions.ACTION_LOOKUP; - } - - public int getMinParams() { - return 0; - } - public int getMaxParams() { - return 0; - } - - @Parameter(names = {ARG_ID}, - description = "ID of the application") - public String id; - - @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT}, - description = "output file for any application report") - public File outputFile; - - @Override - public void validate() throws BadCommandArgumentsException, UsageException { - super.validate(); - if (StringUtils.isEmpty(id)) { - throw new BadCommandArgumentsException("Missing mandatory argument " - + ARG_ID); - } - } - - @Override - public String toString() { - final StringBuilder sb = - new StringBuilder(SliderActions.ACTION_LOOKUP); - if (id!=null) { - sb.append(" "); - sb.append(ARG_ID).append(" ").append(id); - } - if (outputFile != null) { - sb.append(" "); - sb.append(ARG_OUTPUT).append(" ").append(outputFile.getAbsolutePath()); - } - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionNodesArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionNodesArgs.java deleted file mode 100644 index 5a0b019..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionNodesArgs.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.slider.common.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_NODES}, - commandDescription = SliderActions.DESCRIBE_ACTION_NODES) -public class ActionNodesArgs extends AbstractActionArgs { - - /** - * Instance for API use; on CLI the name is derived from {@link #getClusterName()}. - */ - public String instance; - - @Override - public String getActionName() { - return SliderActions.ACTION_NODES; - } - - @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT}, - description = "Output file for the information") - public File outputFile; - - @Parameter(names = {ARG_LABEL}) - public String label = ""; - - @Parameter(names = {ARG_HEALTHY} ) - public boolean healthy; - - @Override - public int getMinParams() { - return 0; - } - - @Override - public int getMaxParams() { - return 1; - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder( - "ActionNodesArgs{"); - sb.append("instance='").append(instance).append('\''); - sb.append(", outputFile=").append(outputFile); - sb.append(", label='").append(label).append('\''); - sb.append(", healthy=").append(healthy); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpgradeArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpgradeArgs.java deleted file mode 100644 index b909cdd..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpgradeArgs.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.slider.common.params; - -import com.beust.jcommander.Parameters; -import org.apache.hadoop.yarn.service.client.params.SliderActions; - -@Parameters(commandNames = { SliderActions.ACTION_UPGRADE }, - commandDescription = SliderActions.DESCRIBE_ACTION_UPGRADE) -public class ActionUpgradeArgs extends AbstractClusterBuildingActionArgs { - - @Override - public String getActionName() { - return SliderActions.ACTION_UPGRADE; - } - -// TODO upgrade container -// @Parameter(names={ARG_CONTAINERS}, variableArity = true, -// description = "stop specific containers") -// public List containers = new ArrayList<>(0); -// -// @Parameter(names={ARG_COMPONENTS}, variableArity = true, -// description = "stop all containers of specific components") -// public List components = new ArrayList<>(0); -// -// @Parameter(names = {ARG_FORCE}, -// description = "force spec upgrade operation") -// public boolean force; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionVersionArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionVersionArgs.java deleted file mode 100644 index b0f17d0..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionVersionArgs.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.slider.common.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 version command - */ -@Parameters(commandNames = { SliderActions.ACTION_VERSION}, - commandDescription = SliderActions.DESCRIBE_ACTION_VERSION) -public class ActionVersionArgs extends AbstractActionArgs { - @Override - public String getActionName() { - return SliderActions.ACTION_VERSION; - } - - public int getMinParams() { - return 0; - } - - /** - * This action does not need hadoop services - * @return false - */ - @Override - public boolean getHadoopServicesRequired() { - return false; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AddonArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AddonArgsDelegate.java deleted file mode 100644 index 3ef8e19..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AddonArgsDelegate.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.slider.common.params; - -import com.beust.jcommander.Parameter; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -public class AddonArgsDelegate extends AbstractArgsDelegate { - - /** - * This is a listing of addon packages - */ - @Parameter(names = {ARG_ADDON}, - arity = 2, - description = "--addon ", - splitter = DontSplitArguments.class) - public List addonTuples = new ArrayList<>(0); - - - /** - * Get the list of addons (may be empty, but never null) - * - * @return map of named addons - * - * @throws BadCommandArgumentsException parse problem - */ - public Map getAddonMap() throws BadCommandArgumentsException { - return convertTupleListToMap("addon", addonTuples); - } - - public List getAddonTuples() { - return addonTuples; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URIArgumentConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URIArgumentConverter.java deleted file mode 100644 index b0d1ebf..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URIArgumentConverter.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.slider.common.params; - -import com.beust.jcommander.converters.BaseConverter; - -import java.net.URI; -import java.net.URISyntaxException; - -public class URIArgumentConverter extends BaseConverter { - - public URIArgumentConverter(String optionName) { - super(optionName); - } - - @Override - public URI convert(String value) { - try { - return new URI(value); - } catch (URISyntaxException e) { - throw new RuntimeException("Cannot make a URI from " + value); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URLArgumentConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URLArgumentConverter.java deleted file mode 100644 index 8894309..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URLArgumentConverter.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.slider.common.params; - -import com.beust.jcommander.converters.BaseConverter; - -import java.net.MalformedURLException; -import java.net.URL; - -public class URLArgumentConverter extends BaseConverter { - public URLArgumentConverter(String optionName) { - super(optionName); - } - - @Override - public URL convert(String value) { - try { - return new URL(value); - } catch (MalformedURLException e) { - throw new RuntimeException("Cannot make a URL from " + value); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java deleted file mode 100644 index 64fd8ae..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java +++ /dev/null @@ -1,611 +0,0 @@ -/* - * 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.slider.common.tools; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; -import org.apache.slider.core.exceptions.BadConfigException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.w3c.dom.Document; -import org.xml.sax.SAXException; - -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; -import javax.xml.parsers.ParserConfigurationException; -import java.io.ByteArrayInputStream; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.StringWriter; -import java.net.MalformedURLException; -import java.net.URL; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import java.util.TreeSet; - -/** - * Methods to aid in config, both in the Configuration class and - * with other parts of setting up Slider-initated processes. - * - * Some of the methods take an argument of a map iterable for their sources; this allows - * the same method - */ -public class ConfigHelper { - private static final Logger log = LoggerFactory.getLogger(ConfigHelper.class); - - /** - * Dump the (sorted) configuration - * @param conf config - * @return the sorted keyset - */ - public static Set dumpConf(Configuration conf) { - Set keys = sortedConfigKeys(conf); - for (String key : keys) { - log.info("{}={}", key, conf.get(key)); - } - return keys; - } - - /** - * Take a configuration and return a sorted set - * @param conf config - * @return the sorted keyset - - */ - public static Set sortedConfigKeys(Iterable> conf) { - TreeSet sorted = new TreeSet(); - for (Map.Entry entry : conf) { - sorted.add(entry.getKey()); - } - return sorted; - } - - /** - * Set an entire map full of values - * - * @param config config to patch - * @param map map of data - * @param origin origin data - */ - public static void addConfigMap(Configuration config, - Map map, - String origin) throws BadConfigException { - addConfigMap(config, map.entrySet(), origin); - } - - /** - * Set an entire map full of values - * - * @param config config to patch - * @param map map of data - * @param origin origin data - */ - public static void addConfigMap(Configuration config, - Iterable> map, - String origin) throws BadConfigException { - for (Map.Entry mapEntry : map) { - String key = mapEntry.getKey(); - String value = mapEntry.getValue(); - if (value == null) { - throw new BadConfigException("Null value for property " + key); - } - config.set(key, value, origin); - } - } - - - /** - * Save a config file in a destination directory on a given filesystem - * @param systemConf system conf used for creating filesystems - * @param confToSave config to save - * @param confdir the directory path where the file is to go - * @param filename the filename - * @return the destination path where the file was saved - * @throws IOException IO problems - */ - public static Path saveConfig(Configuration systemConf, - Configuration confToSave, - Path confdir, - String filename) throws IOException { - FileSystem fs = FileSystem.get(confdir.toUri(), systemConf); - Path destPath = new Path(confdir, filename); - saveConfig(fs, destPath, confToSave); - return destPath; - } - - /** - * Save a config - * @param fs filesystem - * @param destPath dest to save - * @param confToSave config to save - * @throws IOException IO problems - */ - public static void saveConfig(FileSystem fs, - Path destPath, - Configuration confToSave) throws - IOException { - FSDataOutputStream fos = fs.create(destPath); - try { - confToSave.writeXml(fos); - } finally { - IOUtils.closeStream(fos); - } - } - - /** - * Convert to an XML string - * @param conf configuration - * @return conf - * @throws IOException - */ - public static String toXml(Configuration conf) throws IOException { - StringWriter writer = new StringWriter(); - conf.writeXml(writer); - return writer.toString(); - } - - /** - * This will load and parse a configuration to an XML document - * @param fs filesystem - * @param path path - * @return an XML document - * @throws IOException IO failure - */ - public Document parseConfiguration(FileSystem fs, - Path path) throws - IOException { - - - byte[] data = loadBytes(fs, path); - //this is here to track down a parse issue - //related to configurations - String s = new String(data, 0, data.length, "UTF-8"); - log.debug("XML resource {} is \"{}\"", path, s); -/* JDK7 - try (ByteArrayInputStream in = new ByteArrayInputStream(data)) { - Document document = parseConfigXML(in); - return document; - } catch (ParserConfigurationException | SAXException e) { - throw new IOException(e); - } -*/ - ByteArrayInputStream in= null; - try { - in = new ByteArrayInputStream(data); - Document document = parseConfigXML(in); - return document; - } catch (ParserConfigurationException e) { - throw new IOException(e); - } catch (SAXException e) { - throw new IOException(e); - } finally { - IOUtils.closeStream(in); - } - } - - public static byte[] loadBytes(FileSystem fs, Path path) throws IOException { - int len = (int) fs.getLength(path); - byte[] data = new byte[len]; - /* JDK7 - try(FSDataInputStream in = fs.open(path)) { - in.readFully(0, data); - } -*/ - FSDataInputStream in = null; - in = fs.open(path); - try { - in.readFully(0, data); - } finally { - IOUtils.closeStream(in); - } - return data; - } - - /** - * Load a configuration from ANY FS path. The normal Configuration - * loader only works with file:// URIs - * @param fs filesystem - * @param path path - * @return a loaded resource - * @throws IOException - */ - public static Configuration loadConfiguration(FileSystem fs, - Path path) throws IOException { - byte[] data = loadBytes(fs, path); - - ByteArrayInputStream in2; - - in2 = new ByteArrayInputStream(data); - Configuration conf1 = new Configuration(false); - conf1.addResource(in2); - //now clone it while dropping all its sources - Configuration conf2 = new Configuration(false); - String src = path.toString(); - for (Map.Entry entry : conf1) { - String key = entry.getKey(); - String value = entry.getValue(); - conf2.set(key, value, src); - } - return conf2; - } - - - /** - * Generate a config file in a destination directory on the local filesystem - * @param confdir the directory path where the file is to go - * @param filename the filename - * @return the destination path - */ - public static File saveConfig(Configuration generatingConf, - File confdir, - String filename) throws IOException { - - - File destPath = new File(confdir, filename); - OutputStream fos = new FileOutputStream(destPath); - try { - generatingConf.writeXml(fos); - } finally { - IOUtils.closeStream(fos); - } - return destPath; - } - - /** - * Parse an XML Hadoop configuration into an XML document. x-include - * is supported, but as the location isn't passed in, relative - * URIs are out. - * @param in instream - * @return a document - * @throws ParserConfigurationException parser feature problems - * @throws IOException IO problems - * @throws SAXException XML is invalid - */ - public static Document parseConfigXML(InputStream in) throws - ParserConfigurationException, - IOException, - SAXException { - DocumentBuilderFactory docBuilderFactory - = DocumentBuilderFactory.newInstance(); - //ignore all comments inside the xml file - docBuilderFactory.setIgnoringComments(true); - - //allow includes in the xml file - docBuilderFactory.setNamespaceAware(true); - docBuilderFactory.setXIncludeAware(true); - DocumentBuilder builder = docBuilderFactory.newDocumentBuilder(); - return builder.parse(in); - } - - /** - * Load a Hadoop configuration from a local file. - * @param file file to load - * @return a configuration which hasn't actually had the load triggered - * yet. - * @throws FileNotFoundException file is not there - * @throws IOException any other IO problem - */ - public static Configuration loadConfFromFile(File file) throws - IOException { - return loadConfFromFile(file, false); - } - - /** - * - * Load a Hadoop configuration from a local file. - * @param file file to load - * @param loadDefaults flag to indicate if the defaults should be loaded yet - * @return a configuration which hasn't actually had the load triggered - * yet. - * @throws FileNotFoundException file is not there - * @throws IOException any other IO problem - */ - public static Configuration loadConfFromFile(File file, - boolean loadDefaults) throws IOException { - if (!file.exists()) { - throw new FileNotFoundException("File not found :" - + file.getAbsoluteFile()); - } - Configuration conf = new Configuration(loadDefaults); - try { - conf.addResource(file.toURI().toURL()); - } catch (MalformedURLException e) { - // should never happen... - throw new IOException( - "File " + file.toURI() + " doesn't have a valid URL"); - } - return conf; - } - - /** - * Add a configuration from a file to an existing configuration - * @param conf existing configuration - * @param file file to load - * @param overwrite flag to indicate new values should overwrite the predecessor - * @return the merged configuration - * @throws IOException - */ - public static Configuration addConfigurationFile(Configuration conf, - File file, boolean overwrite) - throws IOException { - Configuration c2 = loadConfFromFile(file, false); - mergeConfigurations(conf, c2, file.getAbsolutePath(), overwrite); - return conf; - } - - /** - * Add the system env variables with the given prefix (by convention, env.) - * @param conf existing configuration - * @param prefix prefix - */ - public static void addEnvironmentVariables(Configuration conf, String prefix) { - Map env = System.getenv(); - for (Map.Entry entry : env.entrySet()) { - conf.set(prefix + entry.getKey(),entry.getValue(), "env"); - } - } - - /** - * looks for the config under $confdir/$templateFilename; if not there - * loads it from /conf/templateFile. - * The property {@link SliderKeys#KEY_TEMPLATE_ORIGIN} is set to the - * origin to help debug what's happening - * @param systemConf system conf - * @param confdir conf dir in FS - * @param templateFilename filename in the confdir - * @param fallbackResource resource to fall back on - * @return loaded conf - * @throws IOException IO problems - */ - public static Configuration loadTemplateConfiguration(Configuration systemConf, - Path confdir, - String templateFilename, - String fallbackResource) throws - IOException { - FileSystem fs = FileSystem.get(confdir.toUri(), systemConf); - - Path templatePath = new Path(confdir, templateFilename); - return loadTemplateConfiguration(fs, templatePath, fallbackResource); - } - - /** - * looks for the config under $confdir/$templateFilename; if not there - * loads it from /conf/templateFile. - * The property {@link SliderKeys#KEY_TEMPLATE_ORIGIN} is set to the - * origin to help debug what's happening. - * @param fs Filesystem - * @param templatePath HDFS path for template - * @param fallbackResource resource to fall back on, or "" for no fallback - * @return loaded conf - * @throws IOException IO problems - * @throws FileNotFoundException if the path doesn't have a file and there - * was no fallback. - */ - public static Configuration loadTemplateConfiguration(FileSystem fs, - Path templatePath, - String fallbackResource) - throws IOException { - Configuration conf; - String origin; - if (fs.exists(templatePath)) { - log.debug("Loading template configuration {}", templatePath); - conf = loadConfiguration(fs, templatePath); - origin = templatePath.toString(); - } else { - if (fallbackResource.isEmpty()) { - throw new FileNotFoundException("No config file found at " + templatePath); - } - log.debug("Template {} not found" + - " -reverting to classpath resource {}", templatePath, fallbackResource); - conf = new Configuration(false); - conf.addResource(fallbackResource); - origin = "Resource " + fallbackResource; - } - //force a get - conf.get(SliderXmlConfKeys.KEY_TEMPLATE_ORIGIN); - //now set the origin - conf.set(SliderXmlConfKeys.KEY_TEMPLATE_ORIGIN, origin); - return conf; - } - - - /** - * For testing: dump a configuration - * @param conf configuration - * @return listing in key=value style - */ - public static String dumpConfigToString(Configuration conf) { - Set sorted = sortedConfigKeys(conf); - - StringBuilder builder = new StringBuilder(); - for (String key : sorted) { - - builder.append(key) - .append("=") - .append(conf.get(key)) - .append("\n"); - } - return builder.toString(); - } - - /** - * Merge in one configuration above another - * @param base base config - * @param merge one to merge. This MUST be a non-default-load config to avoid - * merge origin confusion - * @param origin description of the origin for the put operation - * @param overwrite flag to indicate new values should overwrite the predecessor - * @return the base with the merged values - */ - public static Configuration mergeConfigurations(Configuration base, - Iterable> merge, - String origin, - boolean overwrite) { - for (Map.Entry entry : merge) { - String key = entry.getKey(); - if (overwrite || base.get(key) == null) { - base.set(key, entry.getValue(), origin); - } - } - return base; - } - - /** - * Register a resource as a default resource. - * Do not attempt to use this unless you understand that the - * order in which default resources are loaded affects the outcome, - * and that subclasses of Configuration often register new default - * resources - * @param resource the resource name - * @return the URL or null - */ - public static URL registerDefaultResource(String resource) { - URL resURL = getResourceUrl(resource); - if (resURL != null) { - Configuration.addDefaultResource(resource); - } - return resURL; - } - - /** - * Load a configuration from a resource on this classpath. - * If the resource is not found, an empty configuration is returned - * @param resource the resource name - * @return the loaded configuration. - */ - public static Configuration loadFromResource(String resource) { - Configuration conf = new Configuration(false); - URL resURL = getResourceUrl(resource); - if (resURL != null) { - log.debug("loaded resources from {}", resURL); - conf.addResource(resource); - } else{ - log.debug("failed to find {} on the classpath", resource); - } - return conf; - - } - - /** - * Get the URL to a resource, null if not on the CP - * @param resource resource to look for - * @return the URL or null - */ - public static URL getResourceUrl(String resource) { - return ConfigHelper.class.getClassLoader() - .getResource(resource); - } - - /** - * Load a resource that must be on the classpath - * @param resource the resource name - * @return the loaded configuration - * @throws FileNotFoundException if the resource is missing - */ - public static Configuration loadMandatoryResource(String resource) - throws FileNotFoundException { - Configuration conf = new Configuration(false); - URL resURL = getResourceUrl(resource); - if (resURL != null) { - log.debug("loaded resources from {}", resURL); - conf.addResource(resource); - } else { - throw new FileNotFoundException(resource); - } - return conf; - } - - /** - * Propagate a property from a source to a dest config, with a best-effort - * attempt at propagating the origin. - * If the - * @param dest destination - * @param src source - * @param key key to try to copy - * @return true if the key was found and propagated - */ - public static boolean propagate(Configuration dest, - Configuration src, - String key) { - String val = src.get(key); - if (val != null) { - String[] origin = src.getPropertySources(key); - if (origin != null && origin.length > 0) { - dest.set(key, val, origin[0]); - } else { - dest.set(key, val); - return true; - } - } - return false; - } - - - /** - * Take a configuration, return a hash map - * @param conf conf - * @return hash map - */ - public static Map buildMapFromConfiguration(Configuration conf) { - Map map = new HashMap(); - return SliderUtils.mergeEntries(map, conf); - } - - /** - * This goes through the keyset of one configuration and retrieves each value - * from a value source -a different or the same configuration. This triggers - * the property resolution process of the value, resolving any variables against - * in-config or inherited configurations - * @param keysource source of keys - * @param valuesource the source of values - * @return a new configuration where foreach key in keysource, get(key)==valuesource.get(key) - */ - public static Configuration resolveConfiguration( - Iterable> keysource, - Configuration valuesource) { - Configuration result = new Configuration(false); - for (Map.Entry entry : keysource) { - String key = entry.getKey(); - String value = valuesource.get(key); - Preconditions.checkState(value != null, - "no reference for \"%s\" in values", key); - result.set(key, value); - } - return result; - } - - /** - * Register anything we consider deprecated - */ - public static void registerDeprecatedConfigItems() { - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderVersionInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderVersionInfo.java deleted file mode 100644 index 86025ee..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderVersionInfo.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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.slider.common.tools; - -import org.apache.hadoop.util.VersionInfo; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.io.InputStream; -import java.net.URL; -import java.util.Locale; -import java.util.Properties; - -/** - * Extract the version properties, which will look something like - *

- * application.name=${pom.name}
- * application.version=${pom.version}
- * application.build=${buildNumber}
- * application.build.java.version=${java.version}
- * application.build.info=${pom.name}-${pom.version} Built against ${buildNumber} on ${java.version} by ${user.name}
- * 
- * - * the mvn process-resources target will expand the properties - * and add the resources to target/classes, which will then look something like - *
- *   application.name=Slider Core
- *   application.version=0.7.1-SNAPSHOT
- *   application.build=1dd69
- *   application.build.java.version=1.7.0_45
- *   application.build.user=stevel
- *   application.build.info=Slider Core-0.7.1-SNAPSHOT Built against 1dd69 on 1.7.0_45 by stevel
- * 
- * - * Note: the values will change and more properties added. - */ -public class SliderVersionInfo { - private static final Logger log = LoggerFactory.getLogger(SliderVersionInfo.class); - - /** - * Name of the resource containing the filled-in-at-runtime props - */ - public static final String VERSION_RESOURCE = - "org/apache/slider/providers/dynamic/application.properties"; - - public static final String APP_NAME = "application.name"; - public static final String APP_VERSION = "application.version"; - public static final String APP_BUILD = "application.build"; - public static final String APP_BUILD_JAVA_VERSION = "application.build.java.version"; - public static final String APP_BUILD_USER = "application.build.user"; - public static final String APP_BUILD_INFO = "application.build.info"; - public static final String HADOOP_BUILD_INFO = "hadoop.build.info"; - public static final String HADOOP_DEPLOYED_INFO = "hadoop.deployed.info"; - - - public static Properties loadVersionProperties() { - Properties props = new Properties(); - URL resURL = SliderVersionInfo.class.getClassLoader() - .getResource(VERSION_RESOURCE); - assert resURL != null : "Null resource " + VERSION_RESOURCE; - - try { - InputStream inStream = resURL.openStream(); - assert inStream != null : "Null input stream from " + VERSION_RESOURCE; - props.load(inStream); - } catch (IOException e) { - log.warn("IOE loading " + VERSION_RESOURCE, e); - } - return props; - } - - /** - * Load the version info and print it - * @param logger logger - */ - public static void loadAndPrintVersionInfo(Logger logger) { - Properties props = loadVersionProperties(); - logger.info(props.getProperty(APP_BUILD_INFO)); - logger.info("Compiled against Hadoop {}", - props.getProperty(HADOOP_BUILD_INFO)); - logger.info(getHadoopVersionString()); - } - - public static String getHadoopVersionString() { - return String.format(Locale.ENGLISH, - "Hadoop runtime version %s with source checksum %s and build date %s", - VersionInfo.getBranch(), - VersionInfo.getSrcChecksum(), - VersionInfo.getDate()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/buildutils/BuildHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/buildutils/BuildHelper.java deleted file mode 100644 index 80f165f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/buildutils/BuildHelper.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.slider.core.buildutils; - -import org.apache.hadoop.util.VersionInfo; -import org.apache.slider.common.tools.SliderVersionInfo; - -import java.util.Map; -import java.util.Properties; - -/** - * classes to help with the build - */ -public class BuildHelper { - /** - * Add the cluster build information; this will include Hadoop details too - * @param dest map to insert this too - * @param prefix prefix for the build info - */ - public static void addBuildMetadata(Map dest, String prefix) { - - Properties props = SliderVersionInfo.loadVersionProperties(); - dest.put(prefix + "." + SliderVersionInfo.APP_BUILD_INFO, - props.getProperty( - SliderVersionInfo.APP_BUILD_INFO)); - dest.put(prefix + "." + SliderVersionInfo.HADOOP_BUILD_INFO, - props.getProperty(SliderVersionInfo.HADOOP_BUILD_INFO)); - - dest.put(prefix + "." + SliderVersionInfo.HADOOP_DEPLOYED_INFO, - VersionInfo.getBranch() + " @" + VersionInfo.getSrcChecksum()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java deleted file mode 100644 index 9714a0f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java +++ /dev/null @@ -1,344 +0,0 @@ -/* - * 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.slider.core.conf; - -import com.google.common.base.Preconditions; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadConfigException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -/** - * Standard map operations. - * - * This delegates the standard map interface to the map passed in, - * so it can be used to add more actions to the map. - */ -public class MapOperations implements Map { - private static final Logger log = - LoggerFactory.getLogger(MapOperations.class); - public static final String DAYS = ".days"; - public static final String HOURS = ".hours"; - public static final String MINUTES = ".minutes"; - public static final String SECONDS = ".seconds"; - - /** - * Global options - */ - public final Map options; - - public final String name; - - public MapOperations() { - options = new HashMap(); - name = ""; - } - - /** - * Create an instance - * @param name name - * @param options source of options - */ - public MapOperations(String name, Map options) { - Preconditions.checkArgument(options != null, "null map"); - this.options = options; - this.name = name; - } - - /** - * Create an instance from an iterative map entry - * @param entry entry to work with - */ - public MapOperations(Map.Entry> entry) { - Preconditions.checkArgument(entry != null, "null entry"); - this.name = entry.getKey(); - this.options = entry.getValue(); - } - - /** - * Get an option value - * - * @param key key - * @param defVal default value - * @return option in map or the default - */ - public String getOption(String key, String defVal) { - String val = options.get(key); - return val != null ? val : defVal; - } - - /** - * Get a boolean option - * - * @param key option key - * @param defVal default value - * @return option true if the option equals "true", or the default value - * if the option was not defined at all. - */ - public Boolean getOptionBool(String key, boolean defVal) { - String val = getOption(key, Boolean.toString(defVal)); - return Boolean.valueOf(val); - } - - /** - * Get a cluster option or value - * - * @param key option key - * @return the value - * @throws BadConfigException if the option is missing - */ - - public String getMandatoryOption(String key) throws BadConfigException { - String val = options.get(key); - if (val == null) { - if (log.isDebugEnabled()) { - log.debug("Missing key {} from config containing {}", - key, this); - } - String text = "Missing option " + key; - if (SliderUtils.isSet(name)) { - text += " from set " + name; - } - throw new BadConfigException(text); - } - return val; - } - - /** - * Get an integer option; use {@link Integer#decode(String)} so as to take hex - * oct and bin values too. - * - * @param option option name - * @param defVal default value - * @return parsed value - * @throws NumberFormatException if the role could not be parsed. - */ - public int getOptionInt(String option, int defVal) { - String val = getOption(option, Integer.toString(defVal)); - return Integer.decode(val); - } - - /** - * Get a long option; use {@link Long#decode(String)} so as to take hex - * oct and bin values too. - * - * @param option option name - * @param defVal default value - * @return parsed value - * @throws NumberFormatException - */ - public long getOptionLong(String option, long defVal) { - String val = getOption(option, Long.toString(defVal)); - return Long.decode(val); - } - - /** - * Get a mandatory integer option; use {@link Integer#decode(String)} so as to take hex - * oct and bin values too. - * - * @param option option name - * @return parsed value - * @throws NumberFormatException if the option could not be parsed. - * @throws BadConfigException if the option could not be found - */ - public int getMandatoryOptionInt(String option) throws BadConfigException { - getMandatoryOption(option); - return getOptionInt(option, 0); - } - - /** - * Verify that an option is set: that is defined AND non-empty - * @param key - * @throws BadConfigException - */ - public void verifyOptionSet(String key) throws BadConfigException { - if (SliderUtils.isUnset(getOption(key, null))) { - throw new BadConfigException("Unset option %s", key); - } - } - - public void mergeWithoutOverwrite(Map that) { - SliderUtils.mergeMapsIgnoreDuplicateKeys(options, that); - } - - /** - * Merge a map by prefixed keys - * @param that the map to merge in - * @param prefix prefix to match on - * @param overwrite flag to enable overwrite - */ - public void mergeMapPrefixedKeys(Map that, - String prefix, - boolean overwrite) { - for (Map.Entry entry : that.entrySet()) { - String key = entry.getKey(); - if (key.startsWith(prefix)) { - if (overwrite || get(key) == null) { - put(key, entry.getValue()); - } - } - } - } - - /** - * Set a property if it is not already set - * @param key key - * @param value value - */ - public void putIfUnset(String key, String value) { - if (get(key) == null) { - put(key, value); - } - } - - public void set(String key, Object value) { - assert value != null; - put(key, value.toString()); - } - - public int size() { - return options.size(); - } - - public boolean isEmpty() { - return options.isEmpty(); - } - - public boolean containsValue(Object value) { - return options.containsValue(value); - } - - public boolean containsKey(Object key) { - return options.containsKey(key); - } - - public String get(Object key) { - return options.get(key); - } - - public String put(String key, String value) { - return options.put(key, value); - } - - public String remove(Object key) { - return options.remove(key); - } - - public void putAll(Map m) { - options.putAll(m); - } - - public void clear() { - options.clear(); - } - - public Set keySet() { - return options.keySet(); - } - - public Collection values() { - return options.values(); - } - - public Set> entrySet() { - return options.entrySet(); - } - - @SuppressWarnings("EqualsWhichDoesntCheckParameterClass") - public boolean equals(Object o) { - return options.equals(o); - } - - @Override - public int hashCode() { - return options.hashCode(); - } - - public boolean isSet(String key) { - return SliderUtils.isSet(get(key)); - } - - @Override - public String toString() { - StringBuilder builder = new StringBuilder(); - builder.append(name).append("=\n"); - - for (Entry entry : options.entrySet()) { - builder.append(" ") - .append(entry.getKey()) - .append('=') - .append(entry.getValue()) - .append('\n'); - } - return builder.toString(); - } - - /** - * Get the time range of a set of keys - * @param basekey base key to which suffix gets applied - * @param defDays - * @param defHours - * @param defMins - * @param defSecs - * @return the aggregate time range in seconds - */ - public long getTimeRange(String basekey, - int defDays, - int defHours, - int defMins, - int defSecs) { - Preconditions.checkArgument(basekey != null); - int days = getOptionInt(basekey + DAYS, defDays); - int hours = getOptionInt(basekey + HOURS, defHours); - - int minutes = getOptionInt(basekey + MINUTES, defMins); - int seconds = getOptionInt(basekey + SECONDS, defSecs); - // range check - Preconditions.checkState(days >= 0 && hours >= 0 && minutes >= 0 - && seconds >= 0, - "Time range for %s has negative time component %s:%s:%s:%s", - basekey, days, hours, minutes, seconds); - - // calculate total time, schedule the reset if expected - long totalMinutes = (long) days * 24 * 60 + (long) hours * 24 + minutes; - return totalMinutes * 60 + seconds; - } - - /** - * Get all entries with a specific prefix - * @param prefix prefix - * @return a prefixed map, possibly empty - */ - public Map prefixedWith(String prefix) { - - Map prefixed = new HashMap<>(size()); - for (Entry entry: entrySet()) { - if (entry.getKey().startsWith(prefix)) { - prefixed.put(entry.getKey(), entry.getValue()); - } - } - return prefixed; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NoSuchNodeException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NoSuchNodeException.java deleted file mode 100644 index ad2f1a4..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NoSuchNodeException.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.slider.core.exceptions; - -import java.io.IOException; - -/** - * Exception raised when a node cannot be found in the structure - * that is being examined. - */ -public class NoSuchNodeException extends IOException { - - public NoSuchNodeException(String uuid) { - super(uuid); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NotFoundException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NotFoundException.java deleted file mode 100644 index 40cb94d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NotFoundException.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.slider.core.exceptions; - - -/** - * Whatever was being resolved: it was not found - */ -public class NotFoundException extends SliderException { - public NotFoundException(String message, - Object... args) { - super(EXIT_NOT_FOUND, message, args); - } - - public NotFoundException(Throwable throwable, - String message, Object... args) { - super(EXIT_NOT_FOUND, throwable, message, args); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ServiceNotReadyException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ServiceNotReadyException.java deleted file mode 100644 index 435bc1a..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ServiceNotReadyException.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.slider.core.exceptions; - -import java.io.IOException; - -/** - * This is an exception raised when the service does not consider itself - * live (yet) - */ -public class ServiceNotReadyException extends IOException { - - public static final String E_NOT_READY = - "Service not ready for access: please retry"; - - public ServiceNotReadyException(String message) { - super(message); - } - - public ServiceNotReadyException(String message, Throwable cause) { - super(message, cause); - } - - public ServiceNotReadyException(Throwable cause) { - super(cause); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderInternalStateException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderInternalStateException.java deleted file mode 100644 index deddbbc..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderInternalStateException.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.slider.core.exceptions; - -public class SliderInternalStateException extends SliderException { - public SliderInternalStateException(String s) { - super(EXIT_INTERNAL_ERROR, s); - } - - public SliderInternalStateException(String s, Throwable throwable) { - super(EXIT_INTERNAL_ERROR, throwable, s); - } - - public SliderInternalStateException(String message, - Object... args) { - super(EXIT_INTERNAL_ERROR, message, args); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/TriggerClusterTeardownException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/TriggerClusterTeardownException.java deleted file mode 100644 index bb9f430..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/TriggerClusterTeardownException.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.slider.core.exceptions; - -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; - -/** - * An Exception to be thrown for an explicit "shut down the cluster" operation - * raised by the application state or other parts of the AM - */ -public class TriggerClusterTeardownException extends SliderException { - - private final FinalApplicationStatus finalApplicationStatus; - - public TriggerClusterTeardownException(int code, - FinalApplicationStatus finalApplicationStatus, String message, - Object... args) { - super(code, message, args); - this.finalApplicationStatus = finalApplicationStatus; - } - - public FinalApplicationStatus getFinalApplicationStatus() { - return finalApplicationStatus; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UnknownApplicationInstanceException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UnknownApplicationInstanceException.java deleted file mode 100644 index a1f8ae9..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UnknownApplicationInstanceException.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.slider.core.exceptions; - -public class UnknownApplicationInstanceException extends SliderException { - public UnknownApplicationInstanceException(String s) { - super(EXIT_UNKNOWN_INSTANCE, s); - } - - public UnknownApplicationInstanceException(String s, Throwable throwable) { - super(EXIT_UNKNOWN_INSTANCE, throwable, s); - } - - public UnknownApplicationInstanceException(String message, - Object... args) { - super(EXIT_UNKNOWN_INSTANCE, message, args); - } - - /** - * Create an instance with the standard exception name - * @param name name - * @return an instance to throw - */ - public static UnknownApplicationInstanceException unknownInstance(String name) { - return new UnknownApplicationInstanceException(ErrorStrings.E_UNKNOWN_INSTANCE - + ": " + name); - } - public static UnknownApplicationInstanceException unknownInstance(String name, - Throwable throwable) { - UnknownApplicationInstanceException exception = - unknownInstance(name); - exception.initCause(throwable); - return exception; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/WaitTimeoutException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/WaitTimeoutException.java deleted file mode 100644 index 5ad3fdc..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/WaitTimeoutException.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.slider.core.exceptions; - -import java.io.IOException; - -/** - * Called when some spinning operation timed out - */ -public class WaitTimeoutException extends IOException { - public WaitTimeoutException(String message) { - super(message); - } - - public WaitTimeoutException(String message, Throwable cause) { - super(message, cause); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ContainerLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ContainerLauncher.java deleted file mode 100644 index 7e164e4..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ContainerLauncher.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.slider.core.launch; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; -import org.apache.hadoop.yarn.util.ConverterUtils; -import org.apache.slider.common.tools.CoreFileSystem; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.net.InetSocketAddress; - -/** - * Code to ease launching of any container - */ -public class ContainerLauncher extends AbstractLauncher { - private static final Logger log = - LoggerFactory.getLogger(ContainerLauncher.class); - - public ContainerLauncher(Configuration conf, - CoreFileSystem coreFileSystem, - Container container, - Credentials credentials) { - super(coreFileSystem, credentials); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/IrqHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/IrqHandler.java deleted file mode 100644 index 42442d1..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/IrqHandler.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * 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.slider.core.main; - -import sun.misc.Signal; -import sun.misc.SignalHandler; - -import java.io.IOException; - -/** - * This class bundles up all the compiler warnings about abuse of sun.misc - * interrupt handling code - * into one place. - */ -@SuppressWarnings("UseOfSunClasses") -public final class IrqHandler implements SignalHandler { - - public static final String CONTROL_C = "INT"; - public static final String SIGTERM = "TERM"; - - private final String name; - private final Interrupted handler; - - /** - * Create an IRQ handler bound to the specific interrupt - * @param name signal name - * @param handler handler - * @throws IOException - */ - public IrqHandler(String name, Interrupted handler) throws IOException { - this.handler = handler; - this.name = name; - try { - Signal.handle(new Signal(name), this); - } catch (IllegalArgumentException e) { - throw new IOException( - "Could not set handler for signal \"" + name + "\"." - + "This can happen if the JVM has the -Xrs set.", - e); - } - } - - @Override - public String toString() { - return "IrqHandler for signal " + name ; - } - - /** - * Handler for the JVM API for signal handling - * @param signal signal raised - */ -// @Override - public void handle(Signal signal) { - InterruptData data = new InterruptData(signal.getName(), signal.getNumber()); - handler.interrupted(data); - } - - /** - * Interrupt data to pass on. - */ - public static class InterruptData { - public final String name; - public final int number; - - public InterruptData(String name, int number) { - this.name = name; - this.number = number; - } - - @Override - public String toString() { - return "signal " + name + '(' + number + ')'; - } - } - - /** - * Callback on interruption - */ - public interface Interrupted { - - /** - * Handle an interrupt - * @param interruptData data - */ - void interrupted(InterruptData interruptData); - } -} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/RunService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/RunService.java deleted file mode 100644 index c3a1d0e..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/RunService.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.slider.core.main; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.Service; - -/** - * An interface which services can implement to have their - * execution managed by the ServiceLauncher. - * The command line options will be passed down before the - * {@link Service#init(Configuration)} operation is invoked via an - * invocation of {@link RunService#bindArgs(Configuration, String...)} - * After the service has been successfully started via {@link Service#start()} - * the {@link RunService#runService()} method is called to execute the - * service. When this method returns, the service launcher will exit, using - * the return code from the method as its exit option. - */ -public interface RunService extends Service { - - /** - * Propagate the command line arguments. - * This method is called before {@link Service#init(Configuration)}; - * the configuration that is returned from this operation - * is the one that is passed on to the init operation. - * This permits implemenations to change the configuration before - * the init operation.n - * - * - * @param config the initial configuration build up by the - * service launcher. - * @param args argument list list of arguments passed to the command line - * after any launcher-specific commands have been stripped. - * @return the configuration to init the service with. This MUST NOT be null. - * Recommended: pass down the config parameter with any changes - * @throws Exception any problem - */ - Configuration bindArgs(Configuration config, String... args) throws Exception; - - /** - * Run a service. This called after {@link Service#start()} - * @return the exit code - * @throws Throwable any exception to report - */ - int runService() throws Throwable ; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLauncher.java deleted file mode 100644 index f192ec8..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLauncher.java +++ /dev/null @@ -1,642 +0,0 @@ -/* - * 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.slider.core.main; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.service.Service; -import org.apache.hadoop.util.ExitUtil; -import org.apache.hadoop.util.ShutdownHookManager; -import org.apache.hadoop.util.VersionInfo; -import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.net.MalformedURLException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.ListIterator; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * A class to launch any service by name. - * - * It's designed to be subclassed for custom entry points. - * - * - * Workflow - *
    - *
  1. An instance of the class is created
  2. - *
  3. If it implements RunService, it is given the binding args off the CLI
  4. - *
  5. Its service.init() and service.start() methods are called.
  6. - *
  7. If it implements RunService, runService() is called and its return - * code used as the exit code.
  8. - *
  9. Otherwise: it waits for the service to stop, assuming in its start() method - * it begins work
  10. - *
  11. If an exception returned an exit code, that becomes the exit code of the - * command.
  12. - *
- * Error and warning messages are logged to stderr. Why? If the classpath - * is wrong & logger configurations not on it, then no error messages by - * the started app will be seen and the caller is left trying to debug - * using exit codes. - * - */ -@SuppressWarnings("UseOfSystemOutOrSystemErr") -public class ServiceLauncher - implements LauncherExitCodes, IrqHandler.Interrupted, - Thread.UncaughtExceptionHandler { - private static final Logger LOG = LoggerFactory.getLogger( - ServiceLauncher.class); - - protected static final int PRIORITY = 30; - - public static final String NAME = "ServiceLauncher"; - - /** - * Name of the "--conf" argument. - */ - public static final String ARG_CONF = "--conf"; - - public static final String USAGE_MESSAGE = - "Usage: " + NAME + " classname [" + ARG_CONF + - "] | "; - static final int SHUTDOWN_TIME_ON_INTERRUPT = 30 * 1000; - - private volatile S service; - private int serviceExitCode; - @SuppressWarnings("MismatchedQueryAndUpdateOfCollection") - private final List interruptHandlers = new ArrayList<>(1); - private Configuration configuration; - private String serviceClassName; - private static AtomicBoolean signalAlreadyReceived = new AtomicBoolean(false); - - - /** - * Create an instance of the launcher - * @param serviceClassName classname of the service - */ - public ServiceLauncher(String serviceClassName) { - this.serviceClassName = serviceClassName; - } - - /** - * Get the service. Null until and unless - * {@link #launchService(Configuration, String[], boolean)} has completed - * @return the service - */ - public S getService() { - return service; - } - - /** - * Get the configuration constructed from the command line arguments - * @return the configuration used to create the service - */ - public Configuration getConfiguration() { - return configuration; - } - - /** - * The exit code from a successful service execution - * @return the exit code. - */ - public int getServiceExitCode() { - return serviceExitCode; - } - - @Override - public String toString() { - return "ServiceLauncher for " + serviceClassName; - } - - /** - * Launch the service, by creating it, initing it, starting it and then - * maybe running it. {@link RunService#bindArgs(Configuration, String...)} is invoked - * on the service between creation and init. - * - * All exceptions that occur are propagated upwards. - * - * If the method returns a status code, it means that it got as far starting - * the service, and if it implements {@link RunService}, that the - * method {@link RunService#runService()} has completed. - * - * At this point, the service is returned by {@link #getService()}. - * - * @param conf configuration - * @param processedArgs arguments after the configuration parameters - * have been stripped out. - * @param addProcessHooks should process failure handlers be added to - * terminate this service on shutdown. Tests should set this to false. - * @throws ClassNotFoundException classname not on the classpath - * @throws IllegalAccessException not allowed at the class - * @throws InstantiationException not allowed to instantiate it - * @throws InterruptedException thread interrupted - * @throws Throwable any other failure - */ - public int launchService(Configuration conf, - String[] processedArgs, - boolean addProcessHooks) - throws Throwable { - - instantiateService(conf); - - // add any process shutdown hooks - if (addProcessHooks) { - ServiceShutdownHook shutdownHook = new ServiceShutdownHook(service); - ShutdownHookManager.get().addShutdownHook(shutdownHook, PRIORITY); - } - RunService runService = null; - - if (service instanceof RunService) { - //if its a runService, pass in the conf and arguments before init) - runService = (RunService) service; - configuration = runService.bindArgs(configuration, processedArgs); - Preconditions.checkNotNull(configuration, - "null configuration returned by bindArgs()"); - } - - //some class constructors init; here this is picked up on. - if (!service.isInState(Service.STATE.INITED)) { - service.init(configuration); - } - service.start(); - int exitCode = EXIT_SUCCESS; - if (runService != null) { - //assume that runnable services are meant to run from here - exitCode = runService.runService(); - LOG.debug("Service exited with exit code {}", exitCode); - - } else { - //run the service until it stops or an interrupt happens on a different thread. - LOG.debug("waiting for service threads to terminate"); - service.waitForServiceToStop(0); - } - //exit - serviceExitCode = exitCode; - return serviceExitCode; - } - - /** - * Instantiate the service defined in serviceClassName - * . Sets the configuration field - * to the configuration, and service to the service. - * - * @param conf configuration to use - * @throws ClassNotFoundException classname not on the classpath - * @throws IllegalAccessException not allowed at the class - * @throws InstantiationException not allowed to instantiate it - */ - @SuppressWarnings("unchecked") - public Service instantiateService(Configuration conf) - throws ClassNotFoundException, InstantiationException, IllegalAccessException, - ExitUtil.ExitException, NoSuchMethodException, InvocationTargetException { - Preconditions.checkArgument(conf != null, "null conf"); - configuration = conf; - - //Instantiate the class -this requires the service to have a public - // zero-argument constructor - Class serviceClass = - this.getClass().getClassLoader().loadClass(serviceClassName); - Object instance = serviceClass.getConstructor().newInstance(); - if (!(instance instanceof Service)) { - //not a service - throw new ExitUtil.ExitException(EXIT_COMMAND_ARGUMENT_ERROR, - "Not a Service class: " + serviceClassName); - } - - service = (S) instance; - return service; - } - - /** - * Register this class as the handler for the control-C interrupt. - * Can be overridden for testing. - */ - protected void registerInterruptHandler() { - try { - interruptHandlers.add(new IrqHandler(IrqHandler.CONTROL_C, this)); - interruptHandlers.add(new IrqHandler(IrqHandler.SIGTERM, this)); - } catch (IOException e) { - error("Signal handler setup failed : {}" + e, e); - } - } - - /** - * The service has been interrupted -try to shut down the service. - * Give the service time to do this before the exit operation is called - * @param interruptData the interrupted data. - */ - @Override - public void interrupted(IrqHandler.InterruptData interruptData) { - String message = "Service interrupted by " + interruptData.toString(); - warn(message); - if (!signalAlreadyReceived.compareAndSet(false, true)) { - warn("Repeated interrupt: escalating to a JVM halt"); - // signal already received. On a second request to a hard JVM - // halt and so bypass any blocking shutdown hooks. - ExitUtil.halt(EXIT_INTERRUPTED, message); - } - int shutdownTimeMillis = SHUTDOWN_TIME_ON_INTERRUPT; - //start an async shutdown thread with a timeout - ServiceForcedShutdown forcedShutdown = - new ServiceForcedShutdown(shutdownTimeMillis); - Thread thread = new Thread(forcedShutdown); - thread.setDaemon(true); - thread.start(); - //wait for that thread to finish - try { - thread.join(shutdownTimeMillis); - } catch (InterruptedException ignored) { - //ignored - } - if (!forcedShutdown.isServiceStopped()) { - warn("Service did not shut down in time"); - } - exit(EXIT_INTERRUPTED, message); - } - - /** - * Uncaught exception handler. - * If an error is raised: shutdown - * The state of the system is unknown at this point -attempting - * a clean shutdown is dangerous. Instead: exit - * @param thread thread that failed - * @param exception exception - */ - @Override - public void uncaughtException(Thread thread, Throwable exception) { - if (ShutdownHookManager.get().isShutdownInProgress()) { - LOG.error("Thread {} threw an error during shutdown: {}.", - thread.toString(), - exception, - exception); - } else if (exception instanceof Error) { - try { - LOG.error("Thread {} threw an error: {}. Shutting down", - thread.toString(), - exception, - exception); - } catch (Throwable err) { - // We don't want to not exit because of an issue with logging - } - if (exception instanceof OutOfMemoryError) { - // After catching an OOM java says it is undefined behavior, so don't - // even try to clean up or we can get stuck on shutdown. - try { - System.err.println("Halting due to Out Of Memory Error..."); - } catch (Throwable err) { - // Again we don't want to exit because of logging issues. - } - ExitUtil.halt(EXIT_EXCEPTION_THROWN); - } else { - // error other than OutOfMemory - exit(convertToExitException(exception)); - } - } else { - // simple exception in a thread. There's a policy decision here: - // terminate the service vs. keep going after a thread has failed - LOG.error("Thread {} threw an exception: {}", - thread.toString(), - exception, - exception); - } - } - - /** - * Print a warning: currently this goes to stderr - * @param text - */ - protected void warn(String text) { - System.err.println(text); - } - - /** - * Report an error. The message is printed to stderr; the exception - * is logged via the current logger. - * @param message message for the user - * @param thrown the exception thrown - */ - protected void error(String message, Throwable thrown) { - String text = "Exception: " + message; - warn(text); - LOG.error(text, thrown); - } - - /** - * Exit the code. - * This is method can be overridden for testing, throwing an - * exception instead. Any subclassed method MUST raise an - * ExitUtil.ExitException instance. - * The service launcher code assumes that after this method is invoked, - * no other code in the same method is called. - * @param exitCode code to exit - */ - protected void exit(int exitCode, String message) { - ExitUtil.terminate(exitCode, message); - } - - /** - * Exit off an exception. This can be subclassed for testing - * @param ee exit exception - */ - protected void exit(ExitUtil.ExitException ee) { - ExitUtil.terminate(ee.status, ee); - } - - /** - * Get the service name via {@link Service#getName()}. - * If the service is not instantiated, the classname is returned instead. - * @return the service name - */ - public String getServiceName() { - Service s = service; - String name = null; - if (s != null) { - try { - name = s.getName(); - } catch (Exception ignored) { - // ignored - } - } - if (name != null) { - return "service " + name; - } else { - return "service classname " + serviceClassName; - } - } - - /** - * Parse the command line, building a configuration from it, then - * launch the service and wait for it to finish. finally, exit - * passing the status code to the #exit(int) method. - * @param args arguments to the service. arg[0] is - * assumed to be the service classname and is automatically - */ - public void launchServiceAndExit(List args) { - - registerInterruptHandler(); - //Currently the config just the default - Configuration conf = new Configuration(); - String[] processedArgs = extractConfigurationArgs(conf, args); - ExitUtil.ExitException ee = launchServiceRobustly(conf, processedArgs); - System.out.flush(); - System.err.flush(); - exit(ee); - } - - /** - * Extract the configuration arguments and apply them to the configuration, - * building an array of processed arguments to hand down to the service. - * - * @param conf configuration to update - * @param args main arguments. args[0] is assumed to be the service - * classname and is skipped - * @return the processed list. - */ - public static String[] extractConfigurationArgs(Configuration conf, - List args) { - - //convert args to a list - int argCount = args.size(); - if (argCount <= 1 ) { - return new String[0]; - } - List argsList = new ArrayList(argCount); - ListIterator arguments = args.listIterator(); - //skip that first entry - arguments.next(); - while (arguments.hasNext()) { - String arg = arguments.next(); - if (arg.equals(ARG_CONF)) { - //the argument is a --conf file tuple: extract the path and load - //it in as a configuration resource. - - //increment the loop iterator - if (!arguments.hasNext()) { - //overshot the end of the file - exitWithMessage(EXIT_COMMAND_ARGUMENT_ERROR, - ARG_CONF + ": missing configuration file after "); - } - File file = new File(arguments.next()); - if (!file.exists()) { - exitWithMessage(EXIT_COMMAND_ARGUMENT_ERROR, - ARG_CONF + ": configuration file not found: " + file); - } - try { - conf.addResource(file.toURI().toURL()); - } catch (MalformedURLException e) { - LOG.debug("File {} cannot be converted to URL", file, e); - exitWithMessage(EXIT_COMMAND_ARGUMENT_ERROR, - ARG_CONF + ": configuration file path invalid: " + file); - } - } else { - argsList.add(arg); - } - } - String[] processedArgs = new String[argsList.size()]; - argsList.toArray(processedArgs); - return processedArgs; - } - - /** - * Launch a service catching all exceptions and downgrading them to exit codes - * after logging. - * @param conf configuration to use - * @param processedArgs command line after the launcher-specific arguments have - * been stripped out - * @return an exit exception, which will have a status code of 0 if it worked - */ - public ExitUtil.ExitException launchServiceRobustly(Configuration conf, - String[] processedArgs) { - ExitUtil.ExitException exitException; - try { - int exitCode = launchService(conf, processedArgs, true); - if (service != null) { - Throwable failure = service.getFailureCause(); - if (failure != null) { - //the service exited with a failure. - //check what state it is in - Service.STATE failureState = service.getFailureState(); - if (failureState == Service.STATE.STOPPED) { - //the failure occurred during shutdown, not important enough to bother - //the user as it may just scare them - LOG.debug("Failure during shutdown:{} ", failure, failure); - } else { - //throw it for the catch handlers to deal with - throw failure; - } - } - } - exitException = new ExitUtil.ExitException(exitCode, - "In " + serviceClassName); - // either the service succeeded, or an error raised during shutdown, - // which we don't worry that much about - } catch (ExitUtil.ExitException ee) { - exitException = ee; - } catch (Throwable thrown) { - exitException = convertToExitException(thrown); - } - return exitException; - } - - /** - * Convert the exception to one that can be handed off to ExitUtils; - * if it is of the write type it is passed throw as is. If not, a - * new exception with the exit code {@link #EXIT_EXCEPTION_THROWN} - * is created, with the argument thrown as the inner cause - * @param thrown the exception thrown - * @return an exception to terminate the process with - */ - protected ExitUtil.ExitException convertToExitException(Throwable thrown) { - ExitUtil.ExitException exitException; - int exitCode; - String message = thrown.getMessage(); - if (message == null) { - message = thrown.toString(); - } - if (thrown instanceof ExitCodeProvider) { - exitCode = ((ExitCodeProvider) thrown).getExitCode(); - if (LOG.isDebugEnabled()) { - LOG.debug("While running {}: {}", getServiceName(), message, thrown); - } - LOG.error(message); - } else { - // not any of the service launcher exceptions -assume something worse - error(message, thrown); - exitCode = EXIT_EXCEPTION_THROWN; - } - exitException = new ExitUtil.ExitException(exitCode, message); - exitException.initCause(thrown); - return exitException; - } - - - /** - * Build a log message for starting up and shutting down. - * This was grabbed from the ToolRunner code. - * @param classname the class of the server - * @param args arguments - */ - public static String startupShutdownMessage(String classname, - List args) { - final String hostname = NetUtils.getHostname(); - - return toStartupShutdownString("STARTUP_MSG: ", new String[]{ - "Starting " + classname, - " host = " + hostname, - " args = " + args, - " version = " + VersionInfo.getVersion(), - " classpath = " + System.getProperty("java.class.path"), - " build = " + VersionInfo.getUrl() + " -r " - + VersionInfo.getRevision() - + "; compiled by '" + VersionInfo.getUser() - + "' on " + VersionInfo.getDate(), - " java = " + System.getProperty("java.version") - }); - } - - /** - * Exit with a printed message - * @param status status code - * @param message message - */ - private static void exitWithMessage(int status, String message) { - System.err.println(message); - ExitUtil.terminate(status); - } - - private static String toStartupShutdownString(String prefix, String[] msg) { - StringBuilder b = new StringBuilder(prefix); - b.append("\n/************************************************************"); - for (String s : msg) { - b.append("\n").append(prefix).append(s); - } - b.append("\n************************************************************/"); - return b.toString(); - } - - /** - * forced shutdown runnable. - */ - protected class ServiceForcedShutdown implements Runnable { - - private final int shutdownTimeMillis; - private boolean serviceStopped; - - public ServiceForcedShutdown(int shutdownTimeoutMillis) { - this.shutdownTimeMillis = shutdownTimeoutMillis; - } - - @Override - public void run() { - if (service != null) { - service.stop(); - serviceStopped = service.waitForServiceToStop(shutdownTimeMillis); - } else { - serviceStopped = true; - } - } - - private boolean isServiceStopped() { - return serviceStopped; - } - } - - /** - * The real main function, which takes the arguments as a list - * arg 0 must be the service classname - * @param argsList the list of arguments - */ - public static void serviceMain(List argsList) { - if (argsList.isEmpty()) { - exitWithMessage(EXIT_USAGE, USAGE_MESSAGE); - } else { - String serviceClassName = argsList.get(0); - - if (LOG.isDebugEnabled()) { - LOG.debug(startupShutdownMessage(serviceClassName, argsList)); - StringBuilder builder = new StringBuilder(); - for (String arg : argsList) { - builder.append('"').append(arg).append("\" "); - } - LOG.debug(builder.toString()); - } - Thread.setDefaultUncaughtExceptionHandler( - new YarnUncaughtExceptionHandler()); - - ServiceLauncher serviceLauncher = new ServiceLauncher<>(serviceClassName); - serviceLauncher.launchServiceAndExit(argsList); - } - } - - /** - * This is the main entry point for the service launcher. - * @param args command line arguments. - */ - public static void main(String[] args) { - List argsList = Arrays.asList(args); - serviceMain(argsList); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceShutdownHook.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceShutdownHook.java deleted file mode 100644 index de55789..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceShutdownHook.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * 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.slider.core.main; - -import org.apache.hadoop.service.Service; -import org.apache.hadoop.util.ShutdownHookManager; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.lang.ref.WeakReference; - -/** - * JVM Shutdown hook for Service which will stop the - * Service gracefully in case of JVM shutdown. - * This hook uses a weak reference to the service, so - * does not cause services to be retained after they have - * been stopped and deferenced elsewhere. - */ -public class ServiceShutdownHook implements Runnable { - private static final Logger LOG = LoggerFactory.getLogger( - ServiceShutdownHook.class); - - private final WeakReference serviceRef; - private Runnable hook; - - public ServiceShutdownHook(Service service) { - serviceRef = new WeakReference<>(service); - } - - public void register(int priority) { - unregister(); - hook = this; - ShutdownHookManager.get().addShutdownHook(hook, priority); - } - - public synchronized void unregister() { - if (hook != null) { - try { - ShutdownHookManager.get().removeShutdownHook(hook); - } catch (IllegalStateException e) { - LOG.info("Failed to unregister shutdown hook: {}", e, e); - } - hook = null; - } - } - - @Override - public void run() { - Service service; - synchronized (this) { - service = serviceRef.get(); - serviceRef.clear(); - } - if (service == null) { - return; - } - try { - // Stop the Service - service.stop(); - } catch (Throwable t) { - LOG.info("Error stopping {}", service.getName(), t); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/Filenames.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/Filenames.java deleted file mode 100644 index 06ecc51..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/Filenames.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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.slider.core.persist; - -public interface Filenames { - - String RESOURCES = "resources.json"; - String APPCONF = "app_config.json"; - String INTERNAL = "internal.json"; - String WRITELOCK = "writelock"; - String READLOCK = "readlock"; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/LockHeldAction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/LockHeldAction.java deleted file mode 100644 index 6659687..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/LockHeldAction.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.slider.core.persist; - -import org.apache.slider.core.exceptions.SliderException; - -import java.io.IOException; - -/** - * Optional action to add while the lock is held; this is needed to execute - * some other persistent operations within the scope at the same lock - * without inserting too much code into the persister - */ -public interface LockHeldAction { - - /** - * Execute the action - * @throws IOException on any failure - */ - public void execute() throws IOException, SliderException; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/PersistKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/PersistKeys.java deleted file mode 100644 index 1964459..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/PersistKeys.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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.slider.core.persist; - -public class PersistKeys { - - public static final String SCHEMA = - "http://example.org/specification/v2.0.0"; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/SliderRegistryUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/SliderRegistryUtils.java deleted file mode 100644 index ac8fca5..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/SliderRegistryUtils.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.slider.core.registry; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.registry.client.binding.RegistryUtils; -import org.apache.hadoop.yarn.service.conf.SliderKeys; - -/** - * Miscellaneous methods to assist slider registry work - * - */ -public class SliderRegistryUtils { - - - /** - * Get the registry path for an instance under the user's home node - * @param instanceName application instance - * @return a path to the registry location for this application instance. - */ - public static String registryPathForInstance(String instanceName) { - return RegistryUtils.servicePath( - RegistryUtils.currentUser(), SliderKeys.APP_TYPE, instanceName - ); - } - - /** - * Process a path expanding it if needed. - * Validation is delegated to later as the core registry will need - * to do that anyway - * @param path path - * @return a path maybe with some expansion - */ - public static String resolvePath(String path) { - Preconditions.checkArgument(path!=null, "null path"); - Preconditions.checkArgument(!path.isEmpty(), "empty path"); - String newpath = path; - if (path.startsWith("~/")) { - // add user expansion - newpath = RegistryUtils.homePathForCurrentUser() + path.substring(1); - } else if (path.equals("~")) { - newpath = RegistryUtils.homePathForCurrentUser(); - } - return newpath; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/YarnAppListClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/YarnAppListClient.java deleted file mode 100644 index d311fee..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/YarnAppListClient.java +++ /dev/null @@ -1,245 +0,0 @@ -/* - * 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.slider.core.registry; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.slider.client.SliderYarnClientImpl; -import org.apache.slider.api.types.SliderInstanceDescription; -import org.apache.slider.common.tools.CoreFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * Client code for interacting with a list of service instances. - * The initial logic just enumerates service instances in the YARN RM - */ -public class YarnAppListClient { - - private final SliderYarnClientImpl yarnClient; - private final String username; - private final Configuration conf; - private static final Logger log = - LoggerFactory.getLogger(YarnAppListClient.class); - - public YarnAppListClient(SliderYarnClientImpl yarnClient, - String username, - Configuration conf) { - - Preconditions.checkArgument(yarnClient != null, - "yarn client is null: is app inited?"); - Preconditions.checkArgument(username != null, - "username is null"); - Preconditions.checkArgument(conf != null, - "conf parameter is null"); - this.yarnClient = yarnClient; - this.username = username; - this.conf = conf; - } - - /** - * find all live instances of a specific app -if there is more than one - * in the cluster, this returns them all. State should be running or earlier - * in the lifecycle - * @param appname application name - * @return the list of all matching application instances - */ - public List findAllLiveInstances(String appname) - throws YarnException, IOException { - return yarnClient.findAllLiveInstances(username, appname); - } - - - /** - * Find an instance of a application belong to the current user. - * @param appname application name - * @return the app report or null if none is found - * @throws YarnException YARN issues - * @throws IOException IO problems - */ - public ApplicationReport findInstance(String appname) throws - YarnException, - IOException { - return findInstance(appname, null); - } - - /** - * Find an instance of a application belong to the current user in specific - * app states. - * @param appname application name - * @param appStates list of states in which application should be in - * @return the app report or null if none is found - * @throws YarnException YARN issues - * @throws IOException IO problems - */ - public ApplicationReport findInstance(String appname, - EnumSet appStates) - throws YarnException, IOException { - List instances = listInstances(null, appname, appStates); - return yarnClient.findClusterInInstanceList(instances, appname); - } - - /** - * List instances belonging to the specific user - * @return a possibly empty list of AMs - */ - public List listInstances() - throws YarnException, IOException { - return listInstances(null); - } - - /** - * List instances belonging to a specific user - * @return a possibly empty list of AMs - * @param user user if not the default. null means default, "" means all users, - * otherwise it is the name of a user - */ - public List listInstances(String user) - throws YarnException, IOException { - return listInstances(user, null); - } - - /** - * List all instances belonging to a specific user with a specific app name. - * - * @param user - * user if not the default. null means default, "" means all users, - * otherwise it is the name of a user - * @param appName - * application name set as an application tag - * @return a possibly empty list of AMs - * @throws YarnException - * @throws IOException - */ - public List listInstances(String user, String appName) - throws YarnException, IOException { - return listInstances(user, appName, null); - } - - /** - * List all instances belonging to a specific user, with a specific app name - * and in specific app states. - * - * @param user - * user if not the default. null means default, "" means all users, - * otherwise it is the name of a user - * @param appName - * application name set as an application tag - * @param appStates - * a set of application states within which the app should be in - * @return a possibly empty list of AMs - * @throws YarnException - * @throws IOException - */ - public List listInstances(String user, String appName, - EnumSet appStates) - throws YarnException, IOException { - log.debug("listInstances called with user: {}, appName: {}, appStates: {}", - user, appName, appStates); - String listUser = user == null ? username : user; - return yarnClient.listDeployedInstances(listUser, appStates, appName); - } - - /** - * Enumerate slider instances for the current user, and the - * most recent app report, where available. - * @param listOnlyInState boolean to indicate that the instances should - * only include those in a YARN state - * minAppState <= currentState <= maxAppState - * - * @param minAppState minimum application state to include in enumeration. - * @param maxAppState maximum application state to include - * @return a map of application instance name to description - * @throws IOException Any IO problem - * @throws YarnException YARN problems - */ - public Map enumSliderInstances( - boolean listOnlyInState, - YarnApplicationState minAppState, - YarnApplicationState maxAppState) - throws IOException, YarnException { - - CoreFileSystem sliderFileSystem = new CoreFileSystem(conf); - Preconditions.checkArgument(!listOnlyInState || minAppState != null, - "null minAppState when listOnlyInState set"); - Preconditions.checkArgument(!listOnlyInState || maxAppState != null, - "null maxAppState when listOnlyInState set"); - if (!listOnlyInState) { - // if there's not filtering, ask for the entire range of states - minAppState = YarnApplicationState.NEW; - maxAppState = YarnApplicationState.KILLED; - } - // get the complete list of persistent instances - Map persistentInstances = - sliderFileSystem.listPersistentInstances(); - Map descriptions = - new HashMap(persistentInstances.size()); - - if (persistentInstances.isEmpty()) { - // an empty listing is a success if no cluster was named - log.debug("No application instances found"); - return descriptions; - } - - // enum those the RM knows about - List rmInstances = listInstances(); - SliderUtils.sortApplicationsByMostRecent(rmInstances); - Map reportMap = - SliderUtils.buildApplicationReportMap(rmInstances, minAppState, - maxAppState); - log.debug("Persisted {} deployed {} filtered[{}-{}] & de-duped to {}", - persistentInstances.size(), - rmInstances.size(), - minAppState, maxAppState, - reportMap.size()); - - // at this point there is a list of all persistent instances, and - // a (possibly filtered) list of application reports - - for (Map.Entry entry : persistentInstances.entrySet()) { - // loop through the persistent values - String name = entry.getKey(); - - // look up any report from the (possibly filtered) report set - ApplicationReport report = reportMap.get(name); - if (!listOnlyInState || report != null) { - // if the enum wants to filter in state, only add it if there is - // a report in that range. Otherwise: include all values - SliderInstanceDescription sid = new SliderInstanceDescription( - name, entry.getValue(), report); - descriptions.put(name, sid); - } - } - - return descriptions; - - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ExportEntry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ExportEntry.java deleted file mode 100644 index dd6e034..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ExportEntry.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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.slider.core.registry.docstore; - -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.annotate.JsonSerialize; - -/** - * JSON-serializable description of a published key-val configuration. - * - * The values themselves are not serialized in the external view; they have to be served up by the far end - */ -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL) -public class ExportEntry { - - /** - * The value of the export - */ - private String value; - /** - * The container id of the container that is responsible for the export - */ - private String containerId; - /** - * Tag associated with the container - its usually an identifier different than container id - * that allows a soft serial id to all containers of a component - e.g. 1, 2, 3, ... - */ - private String tag; - /** - * An export can be at the level of a component or an application - */ - private String level; - /** - * The time when the export was updated - */ - private String updatedTime; - /** - * The time when the export expires - */ - private String validUntil; - - public ExportEntry() { - } - - public String getValue() { - return value; - } - - public void setValue(String value) { - this.value = value; - } - - public String getContainerId() { - return containerId; - } - - public void setContainerId(String containerId) { - this.containerId = containerId; - } - - public String getTag() { - return tag; - } - - public void setTag(String tag) { - this.tag = tag; - } - - public String getLevel() { - return level; - } - - public void setLevel(String level) { - this.level = level; - } - public String getUpdatedTime() { - return updatedTime; - } - - public void setUpdatedTime(String updatedTime) { - this.updatedTime = updatedTime; - } - - public String getValidUntil() { - return validUntil; - } - - public void setValidUntil(String validUntil) { - this.validUntil = validUntil; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - ExportEntry that = (ExportEntry) o; - - if (value != null ? !value.equals(that.value) : that.value != null) - return false; - return containerId != null ? containerId.equals(that.containerId) : - that.containerId == null; - } - - @Override - public int hashCode() { - int result = value != null ? value.hashCode() : 0; - result = 31 * result + (containerId != null ? containerId.hashCode() : 0); - return result; - } - - @Override - public String toString() { - return new StringBuilder("ExportEntry{"). - append("value='").append(value).append("',"). - append("containerId='").append(containerId).append("',"). - append("tag='").append(tag).append("',"). - append("level='").append(level).append("'"). - append("updatedTime='").append(updatedTime).append("'"). - append("validUntil='").append(validUntil).append("'"). - append(" }").toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigSet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigSet.java deleted file mode 100644 index edc129e..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigSet.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * 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.slider.core.registry.docstore; - -import org.apache.slider.server.appmaster.web.rest.RestPaths; -import org.apache.slider.server.services.utility.PatternValidator; -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.annotate.JsonSerialize; - -import java.util.HashMap; -import java.util.Locale; -import java.util.Map; -import java.util.Set; -import java.util.TreeSet; - -/** - * Represents a set of configurations for an application, component, etc. - * Json serialisable; accessors are synchronized - */ -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL) -public class PublishedConfigSet { - - private static final PatternValidator validator = new PatternValidator( - RestPaths.PUBLISHED_CONFIGURATION_REGEXP); - - public Map configurations = - new HashMap<>(); - - public PublishedConfigSet() { - } - - /** - * Put a name -it will be converted to lower case before insertion. - * Any existing entry will be overwritten (that includes an entry - * with a different case in the original name) - * @param name name of entry - * @param conf configuration - * @throws IllegalArgumentException if not a valid name - */ - public void put(String name, PublishedConfiguration conf) { - String name1 = name.toLowerCase(Locale.ENGLISH); - validateName(name1); - configurations.put(name1, conf); - } - - /** - * Validate the name -restricting it to the set defined in - * {@link RestPaths#PUBLISHED_CONFIGURATION_REGEXP} - * @param name name to validate - * @throws IllegalArgumentException if not a valid name - */ - public static void validateName(String name) { - validator.validate(name); - - } - - public PublishedConfiguration get(String name) { - return configurations.get(name); - } - - public boolean contains(String name) { - return configurations.containsKey(name); - } - - public int size() { - return configurations.size(); - } - - public Set keys() { - TreeSet keys = new TreeSet<>(); - keys.addAll(configurations.keySet()); - return keys; - } - - public PublishedConfigSet shallowCopy() { - PublishedConfigSet that = new PublishedConfigSet(); - for (Map.Entry entry : - configurations.entrySet()) { - that.put(entry.getKey(), entry.getValue().shallowCopy()); - } - return that; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExports.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExports.java deleted file mode 100644 index 58e67ee..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExports.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * 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.slider.core.registry.docstore; - -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.SerializationConfig; -import org.codehaus.jackson.map.annotate.JsonSerialize; - -import java.io.IOException; -import java.util.Date; -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.TreeMap; - -/** - * JSON-serializable description of a published key-val configuration. - * - * The values themselves are not serialized in the external view; they have to be served up by the far end - */ -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL) -public class PublishedExports { - - public String description; - public long updated; - public String updatedTime; - public Map> entries = new HashMap<>(); - - public PublishedExports() { - } - - /** - * build an empty published configuration - * - * @param description configuration description - */ - public PublishedExports(String description) { - this.description = description; - } - - /** - * Build a configuration from the entries - * - * @param description configuration description - * @param entries entries to put - */ - public PublishedExports(String description, - Iterable>> entries) { - this.description = description; - putValues(entries); - } - - /** - * Is the configuration empty. This means either that it has not been given any values, - * or it is stripped down copy - * set down over the wire. - * - * @return true if it is empty - */ - public boolean isEmpty() { - return entries.isEmpty(); - } - - public long getUpdated() { - return updated; - } - - public void setUpdated(long updated) { - this.updated = updated; - this.updatedTime = new Date(updated).toString(); - } - - - public Map> sortedEntries() { - Map> sortedEntries = new TreeMap<>(); - sortedEntries.putAll(entries); - return sortedEntries; - } - - /** - * Set the values from an iterable (this includes a Hadoop Configuration and Java properties - * object). Any existing value set is discarded - * - * @param values values to put - */ - public void putValues(Iterable>> values) { - this.entries = new HashMap<>(); - for (Map.Entry> entry : values) { - this.entries.put(entry.getKey(), entry.getValue()); - } - } - - /** - * Return the values as json string - * - * @return the JSON form - * - * @throws IOException mapping problems - */ - public String asJson() throws IOException { - ObjectMapper mapper = new ObjectMapper(); - mapper.configure(SerializationConfig.Feature.INDENT_OUTPUT, true); - String json = mapper.writeValueAsString(entries); - return json; - } - - /** - * This makes a copy without the nested content -so is suitable for returning as part of the list of a parent's - * values - * - * @return the copy - */ - public PublishedExports shallowCopy() { - PublishedExports that = new PublishedExports(); - that.description = this.description; - that.updated = this.updated; - that.updatedTime = this.updatedTime; - return that; - } - - @Override - public String toString() { - final StringBuilder sb = - new StringBuilder("PublishedConfiguration{"); - sb.append("description='").append(description).append('\''); - sb.append(" entries = ").append(entries.size()); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsOutputter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsOutputter.java deleted file mode 100644 index 67cb094..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsOutputter.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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.slider.core.registry.docstore; - -import com.google.common.base.Charsets; -import com.google.common.base.Preconditions; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; - -/** Output a published configuration */ -public abstract class PublishedExportsOutputter { - - protected final PublishedExports exports; - - protected PublishedExportsOutputter(PublishedExports exports) { - this.exports = exports; - } - - /** - * Create an outputter for the chosen format - * - * @param format format enumeration - * @param exports owning config - * @return the outputter - */ - - public static PublishedExportsOutputter createOutputter(ConfigFormat format, - PublishedExports exports) { - Preconditions.checkNotNull(exports); - switch (format) { - case JSON: - return new JsonOutputter(exports); - default: - throw new RuntimeException("Unsupported format :" + format); - } - } - - public void save(File dest) throws IOException { - FileOutputStream out = null; - try { - out = new FileOutputStream(dest); - save(out); - out.close(); - } finally { - org.apache.hadoop.io.IOUtils.closeStream(out); - } - } - - /** - * Save the content. The default saves the asString() value to the output stream - * - * @param out output stream - * @throws IOException - */ - public void save(OutputStream out) throws IOException { - IOUtils.write(asString(), out, Charsets.UTF_8); - } - - /** - * Convert to a string - * - * @return the string form - * @throws IOException - */ - public abstract String asString() throws IOException; - - public static class JsonOutputter extends PublishedExportsOutputter { - - public JsonOutputter(PublishedExports exports) { - super(exports); - } - - @Override - public void save(File dest) throws IOException { - FileUtils.writeStringToFile(dest, asString(), Charsets.UTF_8); - } - - @Override - public String asString() throws IOException { - return exports.asJson(); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsSet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsSet.java deleted file mode 100644 index 339d3d6..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsSet.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * 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.slider.core.registry.docstore; - -import org.apache.slider.server.appmaster.web.rest.RestPaths; -import org.apache.slider.server.services.utility.PatternValidator; -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.annotate.JsonSerialize; - -import java.util.HashMap; -import java.util.Locale; -import java.util.Map; -import java.util.Set; -import java.util.TreeSet; - -/** - * Represents a set of configurations for an application, component, etc. - * Json serialisable; accessors are synchronized - */ -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL) -public class PublishedExportsSet { - - private static final PatternValidator validator = new PatternValidator( - RestPaths.PUBLISHED_CONFIGURATION_REGEXP); - - public Map exports = new HashMap<>(); - - public PublishedExportsSet() { - } - - /** - * Put a name -it will be converted to lower case before insertion. - * Any existing entry will be overwritten (that includes an entry - * with a different case in the original name) - * @param name name of entry - * @param export published export - * @throws IllegalArgumentException if not a valid name - */ - public void put(String name, PublishedExports export) { - String name1 = name.toLowerCase(Locale.ENGLISH); - validateName(name1); - exports.put(name1, export); - } - - /** - * Validate the name -restricting it to the set defined in - * {@link RestPaths#PUBLISHED_CONFIGURATION_REGEXP} - * @param name name to validate - * @throws IllegalArgumentException if not a valid name - */ - public static void validateName(String name) { - validator.validate(name); - - } - - public PublishedExports get(String name) { - return exports.get(name); - } - - public boolean contains(String name) { - return exports.containsKey(name); - } - - public int size() { - return exports.size(); - } - - public Set keys() { - TreeSet keys = new TreeSet<>(); - keys.addAll(exports.keySet()); - return keys; - } - - public PublishedExportsSet shallowCopy() { - PublishedExportsSet that = new PublishedExportsSet(); - for (Map.Entry entry : exports.entrySet()) { - that.put(entry.getKey(), entry.getValue().shallowCopy()); - } - return that; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/UriMap.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/UriMap.java deleted file mode 100644 index 120966f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/UriMap.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.slider.core.registry.docstore; - -import org.codehaus.jackson.annotate.JsonIgnore; -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.annotate.JsonSerialize; - -import java.util.HashMap; -import java.util.Map; - -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL) -public class UriMap { - - public Map uris = new HashMap<>(); - - @JsonIgnore - public void put(String key, String value) { - uris.put(key, value); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/AMWebClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/AMWebClient.java deleted file mode 100644 index e204178..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/AMWebClient.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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.slider.core.registry.retrieve; - -import com.sun.jersey.api.client.Client; -import com.sun.jersey.api.client.GenericType; -import com.sun.jersey.api.client.WebResource; -import com.sun.jersey.api.client.config.ClientConfig; -import com.sun.jersey.api.client.config.DefaultClientConfig; -import com.sun.jersey.api.json.JSONConfiguration; -import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory; -import com.sun.jersey.client.urlconnection.URLConnectionClientHandler; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.ssl.SSLFactory; -import org.apache.slider.client.rest.BaseRestClient; -import org.apache.slider.core.restclient.HttpVerb; -import org.apache.slider.core.restclient.UgiJerseyBinding; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.net.ssl.HostnameVerifier; -import javax.net.ssl.HttpsURLConnection; -import javax.net.ssl.SSLSocketFactory; -import javax.ws.rs.core.HttpHeaders; -import javax.ws.rs.core.MediaType; -import java.io.IOException; -import java.net.HttpURLConnection; -import java.net.URI; -import java.net.URL; - -/** - * Class to retrieve artifacts from the AM's web site. This sets up - * the redirection and security logic properly - */ -public class AMWebClient { - - - private final BaseRestClient restClient; - private static final Logger - log = LoggerFactory.getLogger(AMWebClient.class); - - - public AMWebClient(Configuration conf) { - UgiJerseyBinding binding = new UgiJerseyBinding(conf); - - restClient = new BaseRestClient(binding.createJerseyClient()); - - } - public WebResource resource(String url) { - return restClient.resource(url); - } - - /** - * Execute the operation. Failures are raised as IOException subclasses - * @param method method to execute - * @param resource resource to work against - * @param c class to build - * @param type expected - * @return an instance of the type T - * @throws IOException on any failure - */ - public T exec(HttpVerb method, WebResource resource, Class c) throws IOException { - return restClient.exec(method, resource, c); - } - - /** - * Execute the operation. Failures are raised as IOException subclasses - * @param method method to execute - * @param resource resource to work against - * @param t type to work with - * @param type expected - * @return an instance of the type T - * @throws IOException on any failure - */ - public T exec(HttpVerb method, WebResource resource, GenericType t) - throws IOException { - return restClient.exec(method, resource, t); - } - - /** - * Execute the GET operation. Failures are raised as IOException subclasses - * @param resource resource to work against - * @param c class to build - * @param type expected - * @return an instance of the type T - * @throws IOException on any failure - */ - public T get(WebResource resource, Class c) throws IOException { - return restClient.get(resource, c); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java deleted file mode 100644 index b0eddb8..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java +++ /dev/null @@ -1,183 +0,0 @@ -/* - * 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.slider.core.registry.retrieve; - -import com.beust.jcommander.Strings; -import com.sun.jersey.api.client.UniformInterfaceException; -import com.sun.jersey.api.client.WebResource; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.registry.client.exceptions.RegistryIOException; -import org.apache.hadoop.registry.client.types.ServiceRecord; -import static org.apache.slider.client.ClientRegistryBinder.*; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.ExceptionConverter; -import org.apache.slider.core.registry.docstore.PublishedConfigSet; -import org.apache.slider.core.registry.docstore.PublishedConfiguration; -import org.apache.slider.core.registry.docstore.PublishedExports; -import org.apache.slider.core.registry.docstore.PublishedExportsSet; -import static org.apache.slider.core.registry.info.CustomRegistryConstants.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.FileNotFoundException; -import java.io.IOException; - -/** - * Registry retriever. - * This hides the HTTP operations that take place to - * get the actual content - */ -public class RegistryRetriever extends AMWebClient { - private static final Logger log = LoggerFactory.getLogger(RegistryRetriever.class); - - private final String externalConfigurationURL; - private final String internalConfigurationURL; - private final String externalExportsURL; - private final String internalExportsURL; - - /** - * Retrieve from a service by locating the - * exported {@link CustomRegistryConstants.PUBLISHER_CONFIGURATIONS_API} - * and working off it. - * - * @param conf configuration to work from - * @param record service record - * @throws RegistryIOException the address type of the endpoint does - * not match that expected (i.e. not a list of URLs), missing endpoint... - */ - public RegistryRetriever(Configuration conf, ServiceRecord record) throws RegistryIOException { - super(conf); - externalConfigurationURL = lookupRestAPI(record, - PUBLISHER_CONFIGURATIONS_API, true); - internalConfigurationURL = lookupRestAPI(record, - PUBLISHER_CONFIGURATIONS_API, false); - externalExportsURL = lookupRestAPI(record, - PUBLISHER_EXPORTS_API, true); - internalExportsURL = lookupRestAPI(record, - PUBLISHER_EXPORTS_API, false); - } - - /** - * Does a bonded registry retriever have a configuration? - * @param external flag to indicate that it is the external entries to fetch - * @return true if there is a URL to the configurations defined - */ - public boolean hasConfigurations(boolean external) { - return !Strings.isStringEmpty( - external ? externalConfigurationURL : internalConfigurationURL); - } - - /** - * Get the configurations of the registry - * @param external flag to indicate that it is the external entries to fetch - * @return the configuration sets - */ - public PublishedConfigSet getConfigurations(boolean external) throws - FileNotFoundException, IOException { - - String confURL = getConfigurationURL(external); - WebResource webResource = resource(confURL); - return get(webResource, PublishedConfigSet.class); - } - - protected String getConfigurationURL(boolean external) throws FileNotFoundException { - String confURL = external ? externalConfigurationURL: internalConfigurationURL; - if (Strings.isStringEmpty(confURL)) { - throw new FileNotFoundException("No configuration URL"); - } - return confURL; - } - - protected String getExportURL(boolean external) throws FileNotFoundException { - String confURL = external ? externalExportsURL: internalExportsURL; - if (Strings.isStringEmpty(confURL)) { - throw new FileNotFoundException("No configuration URL"); - } - return confURL; - } - - /** - * Get the configurations of the registry - * @param external flag to indicate that it is the external entries to fetch - * @return the configuration sets - */ - public PublishedExportsSet getExports(boolean external) throws - FileNotFoundException, IOException { - - String exportsUrl = getExportURL(external); - WebResource webResource = resource(exportsUrl); - return get(webResource, PublishedExportsSet.class); - } - - - /** - * Get a complete configuration, with all values - * @param configSet config set to ask for - * @param name name of the configuration - * @param external flag to indicate that it is an external configuration - * @return the retrieved config - * @throws IOException IO problems - */ - public PublishedConfiguration retrieveConfiguration(PublishedConfigSet configSet, - String name, - boolean external) throws IOException { - String confURL = getConfigurationURL(external); - if (!configSet.contains(name)) { - throw new FileNotFoundException("Unknown configuration " + name); - } - confURL = SliderUtils.appendToURL(confURL, name); - WebResource webResource = resource(confURL); - return get(webResource, PublishedConfiguration.class); - } - - /** - * Get a complete export, with all values - * @param exportSet - * @param name name of the configuration - * @param external flag to indicate that it is an external configuration - * @return the retrieved config - * @throws IOException IO problems - */ - public PublishedExports retrieveExports(PublishedExportsSet exportSet, - String name, - boolean external) throws IOException { - if (!exportSet.contains(name)) { - throw new FileNotFoundException("Unknown export " + name); - } - String exportsURL = getExportURL(external); - exportsURL = SliderUtils.appendToURL(exportsURL, name); - return get(resource(exportsURL), PublishedExports.class); - } - - @Override - public String toString() { - final StringBuilder sb = - new StringBuilder("RegistryRetriever{"); - sb.append("externalConfigurationURL='") - .append(externalConfigurationURL) - .append('\''); - sb.append(", internalConfigurationURL='") - .append(internalConfigurationURL) - .append('\''); - sb.append(", externalExportsURL='").append(externalExportsURL).append('\''); - sb.append(", internalExportsURL='").append(internalExportsURL).append('\''); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/BlockingZKWatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/BlockingZKWatcher.java deleted file mode 100644 index ca49888..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/BlockingZKWatcher.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.slider.core.zk; - -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.net.ConnectException; -import java.util.concurrent.atomic.AtomicBoolean; - -public class BlockingZKWatcher implements Watcher { - - protected static final Logger log = - LoggerFactory.getLogger(BlockingZKWatcher.class); - private final AtomicBoolean connectedFlag = new AtomicBoolean(false); - - @Override - public void process(WatchedEvent event) { - log.info("ZK binding callback received"); - connectedFlag.set(true); - synchronized (connectedFlag) { - try { - connectedFlag.notify(); - } catch (Exception e) { - log.warn("failed while waiting for notification", e); - } - } - } - - /** - * Wait for a flag to go true - * @param timeout timeout in millis - */ - - public void waitForZKConnection(int timeout) - throws InterruptedException, ConnectException { - synchronized (connectedFlag) { - if (!connectedFlag.get()) { - log.info("waiting for ZK event"); - //wait a bit - connectedFlag.wait(timeout); - } - } - if (!connectedFlag.get()) { - throw new ConnectException("Unable to connect to ZK quorum"); - } - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/MiniZooKeeperCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/MiniZooKeeperCluster.java deleted file mode 100644 index 1af883e..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/MiniZooKeeperCluster.java +++ /dev/null @@ -1,402 +0,0 @@ -/* - * 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.slider.core.zk; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.service.AbstractService; -import org.apache.zookeeper.server.NIOServerCnxnFactory; -import org.apache.zookeeper.server.ZooKeeperServer; -import org.apache.zookeeper.server.persistence.FileTxnLog; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.BufferedReader; -import java.io.File; -import java.io.IOException; -import java.io.InputStreamReader; -import java.io.OutputStream; -import java.io.Reader; -import java.net.BindException; -import java.net.InetSocketAddress; -import java.net.Socket; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; - - -/** - * This is a version of the HBase ZK cluster cut out to be standalone. - * - * Important: keep this Java6 language level for now - */ -public class MiniZooKeeperCluster extends AbstractService { - private static final Logger LOG = LoggerFactory.getLogger( - MiniZooKeeperCluster.class); - - private static final int TICK_TIME = 2000; - private static final int CONNECTION_TIMEOUT = 30000; - public static final int MAX_CLIENT_CONNECTIONS = 1000; - - private boolean started; - - /** The default port. If zero, we use a random port. */ - private int defaultClientPort = 0; - - private int clientPort; - - private final List standaloneServerFactoryList; - private final List zooKeeperServers; - private final List clientPortList; - - private int activeZKServerIndex; - private int tickTime = 0; - private File baseDir; - private final int numZooKeeperServers; - private String zkQuorum = ""; - - public MiniZooKeeperCluster(int numZooKeeperServers) { - super("MiniZooKeeperCluster"); - this.numZooKeeperServers = numZooKeeperServers; - this.started = false; - activeZKServerIndex = -1; - zooKeeperServers = new ArrayList(); - clientPortList = new ArrayList(); - standaloneServerFactoryList = new ArrayList(); - } - - - @Override - protected void serviceInit(Configuration conf) throws Exception { - super.serviceInit(conf); - } - - public void setDefaultClientPort(int clientPort) { - if (clientPort <= 0) { - throw new IllegalArgumentException("Invalid default ZK client port: " - + clientPort); - } - this.defaultClientPort = clientPort; - } - - /** - * Selects a ZK client port. Returns the default port if specified. - * Otherwise, returns a random port. The random port is selected from the - * range between 49152 to 65535. These ports cannot be registered with IANA - * and are intended for dynamic allocation (see http://bit.ly/dynports). - */ - private int selectClientPort(Random r) { - if (defaultClientPort > 0) { - return defaultClientPort; - } - return 0xc000 + r.nextInt(0x3f00); - } - - public void setTickTime(int tickTime) { - this.tickTime = tickTime; - } - - public int getBackupZooKeeperServerNum() { - return zooKeeperServers.size() - 1; - } - - public int getZooKeeperServerNum() { - return zooKeeperServers.size(); - } - - // / XXX: From o.a.zk.t.ClientBase - private static void setupTestEnv() { - // during the tests we run with 100K prealloc in the logs. - // on windows systems prealloc of 64M was seen to take ~15seconds - // resulting in test failure (client timeout on first session). - // set env and directly in order to handle static init/gc issues - System.setProperty("zookeeper.preAllocSize", "100"); - FileTxnLog.setPreallocSize(100 * 1024); - } - - @Override - protected void serviceStart() throws Exception { - startup(); - } - - /** - * @return ClientPort server bound to, -1 if there was a - * binding problem and we couldn't pick another port. - * @throws IOException - * @throws InterruptedException - */ - private int startup() throws IOException, - InterruptedException { - if (numZooKeeperServers <= 0) - return -1; - - setupTestEnv(); - started = true; - baseDir = File.createTempFile("zookeeper", ".dir"); - recreateDir(baseDir); - - StringBuilder quorumList = new StringBuilder(); - Random rnd = new Random(); - int tentativePort = selectClientPort(rnd); - - // running all the ZK servers - for (int i = 0; i < numZooKeeperServers; i++) { - File dir = new File(baseDir, "zookeeper_" + i).getAbsoluteFile(); - recreateDir(dir); - int tickTimeToUse; - if (this.tickTime > 0) { - tickTimeToUse = this.tickTime; - } else { - tickTimeToUse = TICK_TIME; - } - ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse); - NIOServerCnxnFactory standaloneServerFactory; - while (true) { - try { - standaloneServerFactory = new NIOServerCnxnFactory(); - standaloneServerFactory.configure( - new InetSocketAddress(tentativePort), - MAX_CLIENT_CONNECTIONS - ); - } catch (BindException e) { - LOG.debug("Failed binding ZK Server to client port: " + - tentativePort, e); - // We're told to use some port but it's occupied, fail - if (defaultClientPort > 0) return -1; - // This port is already in use, try to use another. - tentativePort = selectClientPort(rnd); - continue; - } - break; - } - - // Start up this ZK server - standaloneServerFactory.startup(server); - if (!waitForServerUp(tentativePort, CONNECTION_TIMEOUT)) { - throw new IOException("Waiting for startup of standalone server"); - } - - // We have selected this port as a client port. - clientPortList.add(tentativePort); - standaloneServerFactoryList.add(standaloneServerFactory); - zooKeeperServers.add(server); - if (quorumList.length() > 0) { - quorumList.append(","); - } - quorumList.append("localhost:").append(tentativePort); - tentativePort++; //for the next server - } - - // set the first one to be active ZK; Others are backups - activeZKServerIndex = 0; - - clientPort = clientPortList.get(activeZKServerIndex); - zkQuorum = quorumList.toString(); - LOG.info("Started MiniZK Cluster and connect 1 ZK server " + - "on client port: " + clientPort); - return clientPort; - } - - private void recreateDir(File dir) throws IOException { - if (dir.exists()) { - if (!FileUtil.fullyDelete(dir)) { - throw new IOException("Could not delete zk base directory: " + dir); - } - } - try { - dir.mkdirs(); - } catch (SecurityException e) { - throw new IOException("creating dir: " + dir, e); - } - } - - @Override - protected void serviceStop() throws Exception { - - if (!started) { - return; - } - started = false; - - try { - // shut down all the zk servers - for (int i = 0; i < standaloneServerFactoryList.size(); i++) { - NIOServerCnxnFactory standaloneServerFactory = - standaloneServerFactoryList.get(i); - int clientPort = clientPortList.get(i); - - standaloneServerFactory.shutdown(); - if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) { - throw new IOException("Waiting for shutdown of standalone server"); - } - } - for (ZooKeeperServer zkServer : zooKeeperServers) { - //explicitly close ZKDatabase since ZookeeperServer does not close them - zkServer.getZKDatabase().close(); - } - } finally { - // clear everything - activeZKServerIndex = 0; - standaloneServerFactoryList.clear(); - clientPortList.clear(); - zooKeeperServers.clear(); - } - - LOG.info("Shutdown MiniZK cluster with all ZK servers"); - } - - /**@return clientPort return clientPort if there is another ZK backup can run - * when killing the current active; return -1, if there is no backups. - * @throws IOException - * @throws InterruptedException - */ - public int killCurrentActiveZooKeeperServer() throws IOException, - InterruptedException { - if (!started || activeZKServerIndex < 0) { - return -1; - } - - // Shutdown the current active one - NIOServerCnxnFactory standaloneServerFactory = - standaloneServerFactoryList.get(activeZKServerIndex); - int clientPort = clientPortList.get(activeZKServerIndex); - - standaloneServerFactory.shutdown(); - if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) { - throw new IOException("Waiting for shutdown of standalone server"); - } - - zooKeeperServers.get(activeZKServerIndex).getZKDatabase().close(); - - // remove the current active zk server - standaloneServerFactoryList.remove(activeZKServerIndex); - clientPortList.remove(activeZKServerIndex); - zooKeeperServers.remove(activeZKServerIndex); - LOG.info("Kill the current active ZK servers in the cluster " + - "on client port: " + clientPort); - - if (standaloneServerFactoryList.size() == 0) { - // there is no backup servers; - return -1; - } - clientPort = clientPortList.get(activeZKServerIndex); - LOG.info("Activate a backup zk server in the cluster " + - "on client port: " + clientPort); - // return the next back zk server's port - return clientPort; - } - - /** - * Kill one back up ZK servers - * @throws IOException - * @throws InterruptedException - */ - public void killOneBackupZooKeeperServer() throws IOException, - InterruptedException { - if (!started || activeZKServerIndex < 0 || - standaloneServerFactoryList.size() <= 1) { - return; - } - - int backupZKServerIndex = activeZKServerIndex + 1; - // Shutdown the current active one - NIOServerCnxnFactory standaloneServerFactory = - standaloneServerFactoryList.get(backupZKServerIndex); - int clientPort = clientPortList.get(backupZKServerIndex); - - standaloneServerFactory.shutdown(); - if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) { - throw new IOException("Waiting for shutdown of standalone server"); - } - - zooKeeperServers.get(backupZKServerIndex).getZKDatabase().close(); - - // remove this backup zk server - standaloneServerFactoryList.remove(backupZKServerIndex); - clientPortList.remove(backupZKServerIndex); - zooKeeperServers.remove(backupZKServerIndex); - LOG.info("Kill one backup ZK servers in the cluster " + - "on client port: " + clientPort); - } - - // XXX: From o.a.zk.t.ClientBase - private static boolean waitForServerDown(int port, long timeout) throws - InterruptedException { - long start = System.currentTimeMillis(); - while (true) { - try { - Socket sock = null; - try { - sock = new Socket("localhost", port); - OutputStream outstream = sock.getOutputStream(); - outstream.write("stat".getBytes("UTF-8")); - outstream.flush(); - } finally { - IOUtils.closeSocket(sock); - } - } catch (IOException e) { - return true; - } - - if (System.currentTimeMillis() > start + timeout) { - break; - } - Thread.sleep(250); - } - return false; - } - - // XXX: From o.a.zk.t.ClientBase - private static boolean waitForServerUp(int port, long timeout) throws - InterruptedException { - long start = System.currentTimeMillis(); - while (true) { - try { - Socket sock = null; - sock = new Socket("localhost", port); - BufferedReader reader = null; - try { - OutputStream outstream = sock.getOutputStream(); - outstream.write("stat".getBytes("UTF-8")); - outstream.flush(); - - Reader isr = new InputStreamReader(sock.getInputStream(), "UTF-8"); - reader = new BufferedReader(isr); - String line = reader.readLine(); - if (line != null && line.startsWith("Zookeeper version:")) { - return true; - } - } finally { - IOUtils.closeSocket(sock); - IOUtils.closeStream(reader); - } - } catch (IOException e) { - // ignore as this is expected - LOG.debug("server localhost:" + port + " not up " + e); - } - - if (System.currentTimeMillis() > start + timeout) { - break; - } - Thread.sleep(250); - } - return false; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKCallback.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKCallback.java deleted file mode 100644 index 045b72c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKCallback.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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.slider.core.zk; - -import org.apache.zookeeper.Watcher; - -/** - * Relays ZK watcher events to a closure - */ -public abstract class ZKCallback implements Watcher { - - public ZKCallback() { - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java deleted file mode 100644 index 519cd16..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java +++ /dev/null @@ -1,348 +0,0 @@ -/* - * 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.slider.core.zk; - -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooDefs; -import org.apache.zookeeper.ZooKeeper; -import org.apache.zookeeper.data.ACL; -import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; - - -public class ZKIntegration implements Watcher, Closeable { - -/** - * Base path for services - */ - public static final String ZK_SERVICES = "services"; - /** - * Base path for all Slider references - */ - public static final String ZK_SLIDER = "slider"; - public static final String ZK_USERS = "users"; - public static final String SVC_SLIDER = "/" + ZK_SERVICES + "/" + ZK_SLIDER; - public static final String SVC_SLIDER_USERS = SVC_SLIDER + "/" + ZK_USERS; - - private static final List ZK_USERS_PATH_LIST = new ArrayList(); - static { - ZK_USERS_PATH_LIST.add(ZK_SERVICES); - ZK_USERS_PATH_LIST.add(ZK_SLIDER); - ZK_USERS_PATH_LIST.add(ZK_USERS); - } - - public static final int SESSION_TIMEOUT = 30000; - protected static final Logger log = - LoggerFactory.getLogger(ZKIntegration.class); - private ZooKeeper zookeeper; - private final String username; - private final String clustername; - private final String userPath; - private int sessionTimeout = SESSION_TIMEOUT; - private static final Map ZK_SESSIONS = new HashMap<>(); - -/** - flag to set to indicate that the user path should be created if - it is not already there - */ - private final AtomicBoolean toInit = new AtomicBoolean(false); - private final boolean createClusterPath; - private final Watcher watchEventHandler; - private final String zkConnection; - private final boolean canBeReadOnly; - - protected ZKIntegration(String zkConnection, - String username, - String clustername, - boolean canBeReadOnly, - boolean createClusterPath, - Watcher watchEventHandler, - int sessionTimeout - ) throws IOException { - this.username = username; - this.clustername = clustername; - this.watchEventHandler = watchEventHandler; - this.zkConnection = zkConnection; - this.canBeReadOnly = canBeReadOnly; - this.createClusterPath = createClusterPath; - this.sessionTimeout = sessionTimeout; - this.userPath = mkSliderUserPath(username); - } - - /** - * Returns true only if an active ZK session is available and retrieved from - * cache, false when it has to create a new one. - * - * @return true if from cache, false when new session created - * @throws IOException - */ - public synchronized boolean init() throws IOException { - if (zookeeper != null && getAlive()) { - return true; - } - - synchronized (ZK_SESSIONS) { - if (ZK_SESSIONS.containsKey(zkConnection)) { - zookeeper = ZK_SESSIONS.get(zkConnection); - } - if (zookeeper == null || !getAlive()) { - log.info("Binding ZK client to {}", zkConnection); - zookeeper = new ZooKeeper(zkConnection, sessionTimeout, this, - canBeReadOnly); - ZK_SESSIONS.put(zkConnection, zookeeper); - return false; - } else { - return true; - } - } - } - - /** - * Create an instance bonded to the specific closure - * @param zkConnection - * @param username - * @param clustername - * @param canBeReadOnly - * @param watchEventHandler - * @return the new instance - * @throws IOException - */ - public static ZKIntegration newInstance(String zkConnection, - String username, - String clustername, - boolean createClusterPath, - boolean canBeReadOnly, - Watcher watchEventHandler, - int sessionTimeout) throws IOException { - - return new ZKIntegration(zkConnection, - username, - clustername, - canBeReadOnly, - createClusterPath, - watchEventHandler, - sessionTimeout); - } - - - @Override - public synchronized void close() throws IOException { - if (zookeeper != null) { - try { - zookeeper.close(); - } catch (InterruptedException ignored) { - - } - zookeeper = null; - } - } - - public String getConnectionString() { - return zkConnection; - } - - public String getClusterPath() { - return mkClusterPath(username, clustername); - } - - public boolean getConnected() { - return zookeeper.getState().isConnected(); - } - - public boolean getAlive() { - return zookeeper.getState().isAlive(); - } - - public ZooKeeper.States getState() { - return zookeeper.getState(); - } - - public Stat getClusterStat() throws KeeperException, InterruptedException { - return stat(getClusterPath()); - } - - public boolean exists(String path) throws - KeeperException, - InterruptedException { - return stat(path) != null; - } - - public Stat stat(String path) throws KeeperException, InterruptedException { - return zookeeper.exists(path, false); - } - - @Override - public String toString() { - return "ZK integration bound @ " + zkConnection + ": " + zookeeper; - } - -/** - * Event handler to notify of state events - * @param event - */ - @Override - public void process(WatchedEvent event) { - log.debug("{}", event); - try { - maybeInit(); - } catch (Exception e) { - log.error("Failed to init", e); - } - if (watchEventHandler != null) { - watchEventHandler.process(event); - } - } - - private void maybeInit() throws KeeperException, InterruptedException { - if (!toInit.getAndSet(true) && createClusterPath) { - log.debug("initing"); - //create the user path - mkPath(ZK_USERS_PATH_LIST, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - //create the specific user - createPath(userPath, null, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - } - } - - /** - * Create a path under a parent, don't care if it already exists - * As the path isn't returned, this isn't the way to create sequentially - * numbered nodes. - * @param parent parent dir. Must have a trailing / if entry!=null||empty - * @param entry entry -can be null or "", in which case it is not appended - * @param acl - * @param createMode - * @return the path if created; null if not - */ - public String createPath(String parent, - String entry, - List acl, - CreateMode createMode) throws KeeperException, InterruptedException { - //initial create of full path - assert acl != null; - assert !acl.isEmpty(); - assert parent != null; - String path = parent; - if (entry != null) { - path = path + entry; - } - try { - log.debug("Creating ZK path {}", path); - return zookeeper.create(path, null, acl, createMode); - } catch (KeeperException.NodeExistsException ignored) { - //node already there - log.debug("node already present:{}",path); - return null; - } - } - - /** - * Recursive path create - * @param paths path list - * @param acl acl list - * @param createMode create modes - */ - public void mkPath(List paths, - List acl, - CreateMode createMode) throws KeeperException, InterruptedException { - String history = "/"; - for (String entry : paths) { - createPath(history, entry, acl, createMode); - history = history + entry + "/"; - } - } - - /** - * Delete a node, does not throw an exception if the path is not fond - * @param path path to delete - * @return true if the path could be deleted, false if there was no node to delete - * - */ - public boolean delete(String path) throws - InterruptedException, - KeeperException { - try { - zookeeper.delete(path, -1); - log.debug("Deleting {}", path); - return true; - } catch (KeeperException.NoNodeException ignored) { - return false; - } - } - - /** - * Recursively delete a node, does not throw exception if any node does not exist. - * @param path - * @return true if delete was successful - */ - public boolean deleteRecursive(String path) throws KeeperException, InterruptedException { - - try { - List children = zookeeper.getChildren(path, false); - for (String child : children) { - deleteRecursive(path + "/" + child); - } - delete(path); - } catch (KeeperException.NoNodeException ignored) { - return false; - } - - return true; - } - - /** - * Build the path to a cluster; exists once the cluster has come up. - * Even before that, a ZK watcher could wait for it. - * @param username user - * @param clustername name of the cluster - * @return a strin - */ - public static String mkClusterPath(String username, String clustername) { - return mkSliderUserPath(username) + "/" + clustername; - } -/** - * Build the path to a cluster; exists once the cluster has come up. - * Even before that, a ZK watcher could wait for it. - * @param username user - * @return a string - */ - public static String mkSliderUserPath(String username) { - return SVC_SLIDER_USERS + "/" + username; - } - - /** - * Blocking enum of users. - * @return an unordered list of clusters under a user - */ - public List getClusters() throws KeeperException, - InterruptedException { - return zookeeper.getChildren(userPath, null); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKPathBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKPathBuilder.java deleted file mode 100644 index b088568..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKPathBuilder.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * 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.slider.core.zk; - -import java.util.Locale; - -public final class ZKPathBuilder { - - private final String username, appname, clustername; - private final String quorum; - - private String appPath; - private String registryPath; - private final String appQuorum; - - public ZKPathBuilder(String username, - String appname, - String clustername, - String quorum, - String appQuorum) { - this.username = username; - this.appname = appname; - this.clustername = clustername; - this.quorum = quorum; - appPath = buildAppPath(); - registryPath = buildRegistryPath(); - this.appQuorum = appQuorum; - } - - public String buildAppPath() { - return String.format(Locale.ENGLISH, "/yarnapps_%s_%s_%s", appname, - username, clustername); - - } - - public String buildRegistryPath() { - return String.format(Locale.ENGLISH, "/services_%s_%s_%s", appname, - username, clustername); - - } - - public String getQuorum() { - return quorum; - } - - public String getAppQuorum() { - return appQuorum; - } - - public String getAppPath() { - return appPath; - } - - public void setAppPath(String appPath) { - this.appPath = appPath; - } - - public String getRegistryPath() { - return registryPath; - } - - public void setRegistryPath(String registryPath) { - this.registryPath = registryPath; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java deleted file mode 100644 index 1e1b1b8..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * 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.slider.providers; - -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.util.StringUtils; -import org.apache.hadoop.yarn.api.ApplicationConstants; -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.service.conf.SliderKeys; -import org.apache.hadoop.yarn.service.provider.ProviderService; -import org.apache.hadoop.yarn.service.provider.ProviderUtils; -import org.apache.hadoop.yarn.service.timelineservice.ServiceTimelinePublisher; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.api.resource.ContainerState; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.launch.CommandLineBuilder; -import org.apache.slider.core.launch.ContainerLauncher; -import org.apache.slider.core.registry.docstore.PublishedConfiguration; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.state.StateAccessForProviders; -import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Map; -import java.util.Map.Entry; - -import static org.apache.hadoop.yarn.service.utils.ServiceApiUtil.$; - -public abstract class AbstractProviderService extends AbstractService - implements ProviderService, SliderKeys { - - protected static final Logger log = - LoggerFactory.getLogger(AbstractProviderService.class); - private static final ProviderUtils providerUtils = new ProviderUtils(); - protected StateAccessForProviders amState; - protected YarnRegistryViewForProviders yarnRegistry; - private ServiceTimelinePublisher serviceTimelinePublisher; - - protected AbstractProviderService(String name) { - super(name); - } - - public abstract void processArtifact(ContainerLauncher launcher, - Application application, RoleInstance roleInstance, - SliderFileSystem fileSystem) throws IOException; - - - public void buildContainerLaunchContext(ContainerLauncher launcher, - Application application, Container container, ProviderRole providerRole, - SliderFileSystem fileSystem, RoleInstance roleInstance) - throws IOException, SliderException { - Component component = providerRole.component; - processArtifact(launcher, application, roleInstance, fileSystem); - - // Generate tokens (key-value pair) for config substitution. - // Get pre-defined tokens - Map globalTokens = amState.getGlobalSubstitutionTokens(); - Map tokensForSubstitution = providerUtils - .initCompTokensForSubstitute(null); - tokensForSubstitution.putAll(globalTokens); - // Set the environment variables in launcher - launcher.putEnv(SliderUtils - .buildEnvMap(component.getConfiguration(), tokensForSubstitution)); - launcher.setEnv("WORK_DIR", ApplicationConstants.Environment.PWD.$()); - launcher.setEnv("LOG_DIR", ApplicationConstants.LOG_DIR_EXPANSION_VAR); - if (System.getenv(HADOOP_USER_NAME) != null) { - launcher.setEnv(HADOOP_USER_NAME, System.getenv(HADOOP_USER_NAME)); - } - launcher.setEnv("LANG", "en_US.UTF-8"); - launcher.setEnv("LC_ALL", "en_US.UTF-8"); - launcher.setEnv("LANGUAGE", "en_US.UTF-8"); - - for (Entry entry : launcher.getEnv().entrySet()) { - tokensForSubstitution.put($(entry.getKey()), entry.getValue()); - } - providerUtils.addComponentHostTokens(tokensForSubstitution, amState); - - // create config file on hdfs and add local resource - - // substitute launch command - String launchCommand = ProviderUtils - .substituteStrWithTokens(component.getLaunchCommand(), - tokensForSubstitution); - CommandLineBuilder operation = new CommandLineBuilder(); - operation.add(launchCommand); - operation.addOutAndErrFiles(OUT_FILE, ERR_FILE); - launcher.addCommand(operation.build()); - - // publish exports - providerUtils - .substituteMapWithTokens(application.getQuicklinks(), tokensForSubstitution); - PublishedConfiguration pubconf = new PublishedConfiguration(QUICK_LINKS, - application.getQuicklinks().entrySet()); - amState.getPublishedSliderConfigurations().put(QUICK_LINKS, pubconf); - if (serviceTimelinePublisher != null) { - serviceTimelinePublisher.serviceAttemptUpdated(application); - } - } - - public boolean processContainerStatus(ContainerId containerId, - ContainerStatus status) { - log.debug("Handling container status: {}", status); - if (SliderUtils.isEmpty(status.getIPs()) || - SliderUtils.isUnset(status.getHost())) { - return true; - } - RoleInstance instance = amState.getOwnedContainer(containerId); - if (instance == null) { - // container is completed? - return false; - } - - // TODO publish ip and host - org.apache.slider.api.resource.Container container = - instance.providerRole.component.getContainer(containerId.toString()); - if (container != null) { - container.setIp(StringUtils.join(",", status.getIPs())); - container.setHostname(status.getHost()); - container.setState(ContainerState.READY); - } else { - log.warn(containerId + " not found in Application!"); - } - return false; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/MonitorDetail.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/MonitorDetail.java deleted file mode 100644 index 27d3415..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/MonitorDetail.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.slider.providers; - -/** - * Details about some exported information from a provider to the AM web UI. - */ -public class MonitorDetail { - - private final String value; - private final boolean isUrl; - - public MonitorDetail(String value, boolean isUrl) { - this.value = value; - this.isUrl = isUrl; - } - - public String getValue() { - return value; - } - - public boolean isUrl() { - return isUrl; - } - - public String toString() { - return "MonitorDetail[" + value + " isUrl=" + isUrl + "]"; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicy.java deleted file mode 100644 index 128dd5d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicy.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * 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.slider.providers; - -/** - * Placement values. - * This is nominally a bitmask, though not all values make sense - */ -public class PlacementPolicy { - - /** - * Default value: history used, anti-affinity hinted at on rebuild/flex up - */ - public static final int NONE = 0; - - /** - * Default value: history used, anti-affinity hinted at on rebuild/flex up - */ - public static final int DEFAULT = NONE; - - /** - * Strict placement: when asking for an instance for which there is - * history, mandate that it is strict - */ - public static final int STRICT = 1; - - /** - * No data locality; do not use placement history - */ - public static final int ANYWHERE = 2; - - /** - * @Deprecated: use {@link #ANYWHERE} - */ - @Deprecated - public static final int NO_DATA_LOCALITY = ANYWHERE; - - /** - * Anti-affinity is mandatory. - */ - public static final int ANTI_AFFINITY_REQUIRED = 4; - - /** - * Exclude from flexing; used internally to mark AMs. - */ - public static final int EXCLUDE_FROM_FLEXING = 16; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicyOptions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicyOptions.java deleted file mode 100644 index e61f944..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicyOptions.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * 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.slider.providers; - -public enum PlacementPolicyOptions { - - EXCLUDE_FROM_FLEXING, - NO_DATA_LOCALITY, - ANTI_AFFINITY_REQUIRED, -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompleted.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompleted.java deleted file mode 100644 index f6ff4fd..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompleted.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.slider.providers; - -/** - * This is the callback triggered by the {@link ProviderCompletedCallable} - * when it generates a notification - */ -public interface ProviderCompleted { - - public void eventCallbackEvent(Object parameter); - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompletedCallable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompletedCallable.java deleted file mode 100644 index 47939c9..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompletedCallable.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.slider.providers; - -import java.util.concurrent.Callable; - -public class ProviderCompletedCallable implements Callable { - - private final ProviderCompleted callback; - private final Object parameter; - - public ProviderCompletedCallable(ProviderCompleted callback, Object parameter) { - this.callback = callback; - this.parameter = parameter; - } - - @Override - public Object call() throws Exception { - callback.eventCallbackEvent(parameter); - return parameter; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCore.java deleted file mode 100644 index b07fc29..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCore.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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.slider.providers; - -import org.apache.hadoop.conf.Configuration; - -import java.util.List; -public interface ProviderCore { - - String getName(); - - List getRoles(); - - Configuration getConf(); -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderRole.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderRole.java deleted file mode 100644 index 6fd85bf..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderRole.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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.slider.providers; - -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.api.resource.Component; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.servicemonitor.MonitorUtils; -import org.apache.slider.server.servicemonitor.Probe; - -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.atomic.AtomicLong; - -/** - * Provider role and key for use in app requests. - * - * This class uses the role name as the key for hashes and in equality tests, - * and ignores the other values. - */ -public final class ProviderRole { - public final String name; - public final int id; - public int placementPolicy; - public int nodeFailureThreshold; - public final long placementTimeoutSeconds; - public final String labelExpression; - public final Component component; - public AtomicLong componentIdCounter = null; - public Queue failedInstances = new ConcurrentLinkedQueue<>(); - public Probe probe; - - public ProviderRole(String name, int id) { - this(name, - id, - PlacementPolicy.DEFAULT, - ResourceKeys.DEFAULT_NODE_FAILURE_THRESHOLD, - ResourceKeys.DEFAULT_PLACEMENT_ESCALATE_DELAY_SECONDS, - ResourceKeys.DEF_YARN_LABEL_EXPRESSION); - } - - /** - * Create a provider role - * @param name role/component name - * @param id ID. This becomes the YARN priority - * @param policy placement policy - * @param nodeFailureThreshold threshold for node failures (within a reset interval) - * after which a node failure is considered an app failure - * @param placementTimeoutSeconds for lax placement, timeout in seconds before - * @param labelExpression label expression for requests; may be null - */ - public ProviderRole(String name, - int id, - int policy, - int nodeFailureThreshold, - long placementTimeoutSeconds, - String labelExpression) { - this(name, - id, - policy, - nodeFailureThreshold, - placementTimeoutSeconds, - labelExpression, - new Component().name(name).numberOfContainers(0L)); - } - - /** - * Create a provider role with a role group - * @param name role/component name - * @param id ID. This becomes the YARN priority - * @param policy placement policy - * @param nodeFailureThreshold threshold for node failures (within a reset interval) - * after which a node failure is considered an app failure - * @param placementTimeoutSeconds for lax placement, timeout in seconds before - * @param labelExpression label expression for requests; may be null - */ - public ProviderRole(String name, int id, int policy, - int nodeFailureThreshold, long placementTimeoutSeconds, - String labelExpression, Component component) { - this.name = name; - this.id = id; - this.placementPolicy = policy; - this.nodeFailureThreshold = nodeFailureThreshold; - this.placementTimeoutSeconds = placementTimeoutSeconds; - this.labelExpression = labelExpression; - this.component = component; - if(component.getUniqueComponentSupport()) { - componentIdCounter = new AtomicLong(0); - } - this.probe = MonitorUtils.getProbe(component.getReadinessCheck()); - } - - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - ProviderRole that = (ProviderRole) o; - return name.equals(that.name); - } - - @Override - public int hashCode() { - return name.hashCode(); - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder("ProviderRole{"); - sb.append("name='").append(name).append('\''); - sb.append(", id=").append(id); - sb.append(", placementPolicy=").append(placementPolicy); - sb.append(", nodeFailureThreshold=").append(nodeFailureThreshold); - sb.append(", placementTimeoutSeconds=").append(placementTimeoutSeconds); - sb.append(", labelExpression='").append(labelExpression).append('\''); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/AppMasterActionOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/AppMasterActionOperations.java deleted file mode 100644 index 288f25a..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/AppMasterActionOperations.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.slider.server.appmaster; - -import org.apache.slider.server.appmaster.operations.RMOperationHandlerActions; - -/** - * Interface of AM operations - */ -public interface AppMasterActionOperations extends RMOperationHandlerActions { - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PrivilegedConnectToCM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PrivilegedConnectToCM.java deleted file mode 100644 index 65b88cf..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PrivilegedConnectToCM.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.slider.server.appmaster; - - -import org.apache.hadoop.yarn.api.ContainerManagementProtocol; - -import java.net.InetSocketAddress; -import java.security.PrivilegedAction; - -/** - * Implement privileged connection to the CM - * - */ -public class PrivilegedConnectToCM implements PrivilegedAction { - final SliderAppMaster appMaster; - final InetSocketAddress cmAddress; - - public PrivilegedConnectToCM(SliderAppMaster appMaster, - InetSocketAddress cmAddress) { - this.appMaster = appMaster; - this.cmAddress = cmAddress; - } - - - @Override //PrivilegedAction - public ContainerManagementProtocol run() { - return ((ContainerManagementProtocol) appMaster.getProxy( - ContainerManagementProtocol.class, - cmAddress)); - } -} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/ProtobufClusterServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/ProtobufClusterServices.java deleted file mode 100644 index 5d52441..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/ProtobufClusterServices.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.slider.server.appmaster; - -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.util.Records; -import org.apache.hadoop.yarn.util.resource.Resources; -import org.apache.slider.server.appmaster.state.AbstractClusterServices; - -public class ProtobufClusterServices extends AbstractClusterServices { - - public Resource newResource() { - return Records.newRecord(Resource.class); - } - - @Override - public Resource newResource(int memory, int cores) { - return Resources.createResource(memory, cores); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PublishedArtifacts.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PublishedArtifacts.java deleted file mode 100644 index fdc386f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PublishedArtifacts.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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.slider.server.appmaster; - -/** - * This is the name of YARN artifacts that are published - */ -public interface PublishedArtifacts { - - String COMPLETE_CONFIG = "complete-config"; - String CORE_SITE_CONFIG = "core-site"; - String HDFS_SITE_CONFIG = "hdfs-site"; - String YARN_SITE_CONFIG = "yarn-site"; - String LOG4J = "log4j"; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/RoleLaunchService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/RoleLaunchService.java deleted file mode 100644 index d96d13e..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/RoleLaunchService.java +++ /dev/null @@ -1,168 +0,0 @@ -/* - * 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.slider.server.appmaster; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.core.launch.ContainerLauncher; -import org.apache.hadoop.yarn.service.provider.ProviderService; -import org.apache.hadoop.yarn.service.provider.ProviderFactory; -import org.apache.slider.server.appmaster.actions.QueueAccess; -import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; -import org.apache.slider.server.appmaster.state.ContainerAssignment; -import org.apache.slider.server.services.workflow.ServiceThreadFactory; -import org.apache.slider.server.services.workflow.WorkflowExecutorService; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -import static org.apache.hadoop.yarn.service.conf.SliderKeys.KEY_CONTAINER_LAUNCH_DELAY; - -/** - * A service for launching containers - */ -public class RoleLaunchService - extends WorkflowExecutorService { - protected static final Logger log = - LoggerFactory.getLogger(RoleLaunchService.class); - - public static final String ROLE_LAUNCH_SERVICE = "RoleLaunchService"; - - - /** - * Queue submission API - */ - private QueueAccess actionQueue; - - /** - * Filesystem to use for the launch - */ - private SliderFileSystem fs; - - - private Map envVars = new HashMap<>(); - - /** - * Construct an instance of the launcher - * @param queueAccess - * @param fs filesystem - * @param envVars environment variables - */ - public RoleLaunchService(QueueAccess queueAccess, SliderFileSystem fs, - Map envVars) { - super(ROLE_LAUNCH_SERVICE); - this.actionQueue = queueAccess; - this.fs = fs; - this.envVars = envVars; - } - - public RoleLaunchService(SliderFileSystem fs) { - super(ROLE_LAUNCH_SERVICE); - this.fs = fs; - } - - @Override - public void init(Configuration conf) { - super.init(conf); - setExecutor(Executors.newCachedThreadPool( - new ServiceThreadFactory(ROLE_LAUNCH_SERVICE, true))); - } - - /** - * Start an asychronous launch operation - * @param assignment container assignment - * @param credentials credentials to use - */ - public void launchRole(ContainerAssignment assignment, - Application application, Credentials credentials) { - } - - public void launchComponent(Application application, - ComponentInstance instance, Container container) { - RoleLaunchService.RoleLauncher launcher = - new RoleLaunchService.RoleLauncher(application, instance, - container); - execute(launcher); - } - - /** - * Thread that runs on the AM to launch a container - */ - private class RoleLauncher implements Runnable { - // Allocated container - public final Container container; - public final Application application; - public ComponentInstance instance; - - public RoleLauncher( - Application application, - ComponentInstance instance, Container container) { - this.container = container; - this.application = application; - this.instance = instance; - } - - @Override - public void run() { - try { - ContainerLauncher containerLauncher = - new ContainerLauncher(null, fs, container, null); - containerLauncher.putEnv(envVars); - - Component compSpec = instance.getCompSpec(); - ProviderService provider = ProviderFactory.getProviderService( - compSpec.getArtifact()); - provider.buildContainerLaunchContext(containerLauncher, application, - instance, fs, getConfig()); - - long delay = compSpec.getConfiguration() - .getPropertyLong(KEY_CONTAINER_LAUNCH_DELAY, 0); - long maxDelay = getConfig() - .getLong(YarnConfiguration.RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS, - YarnConfiguration.DEFAULT_RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS); - if (delay > maxDelay/1000) { - log.warn("Container launch delay of {} exceeds the maximum allowed of" - + " {} seconds. Delay will not be utilized.", - delay, maxDelay/1000); - delay = 0; - } - if (delay > 0) { - Thread.sleep(delay * 1000); - } - instance.getComponent().getScheduler().getNmClient() - .startContainerAsync(container, - containerLauncher.completeContainerLaunch()); - } catch (Exception e) { - log.error("Exception thrown while trying to start " + instance - .getCompInstanceName() - + " container = " + container.getId() + " on host " + container - .getNodeId(), e); - } - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java deleted file mode 100644 index 06dde67..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java +++ /dev/null @@ -1,2138 +0,0 @@ -/* - * 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.slider.server.appmaster; - -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.health.HealthCheckRegistry; -import com.google.common.base.Preconditions; -import com.google.protobuf.BlockingService; -import org.apache.commons.collections.CollectionUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.HdfsConfiguration; -import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; -import org.apache.hadoop.http.HttpConfig; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; -import org.apache.hadoop.registry.client.api.RegistryOperations; -import org.apache.hadoop.registry.client.binding.RegistryPathUtils; -import org.apache.hadoop.registry.client.binding.RegistryTypeUtils; -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.registry.server.integration.RMRegistryOperationsService; -import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.service.Service; -import org.apache.hadoop.service.ServiceOperations; -import org.apache.hadoop.service.ServiceStateChangeListener; -import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; -import org.apache.hadoop.yarn.api.records.ApplicationAccessType; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.ContainerState; -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.NodeState; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.hadoop.yarn.client.api.TimelineV2Client; -import org.apache.hadoop.yarn.client.api.YarnClient; -import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; -import org.apache.hadoop.yarn.client.api.async.NMClientAsync; -import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException; -import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.ipc.YarnRPC; -import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; -import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager; -import org.apache.hadoop.yarn.service.provider.ProviderService; -import org.apache.hadoop.yarn.service.provider.ProviderFactory; -import org.apache.hadoop.yarn.util.ConverterUtils; -import org.apache.hadoop.yarn.webapp.WebAppException; -import org.apache.hadoop.yarn.webapp.WebApps; -import org.apache.hadoop.yarn.webapp.util.WebAppUtils; -import org.apache.slider.api.InternalKeys; -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.api.RoleKeys; -import org.apache.slider.api.proto.Messages; -import org.apache.slider.api.proto.SliderClusterAPI; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.hadoop.yarn.service.conf.SliderExitCodes; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; -import org.apache.hadoop.yarn.service.client.params.SliderAMArgs; -import org.apache.hadoop.yarn.service.client.params.SliderAMCreateAction; -import org.apache.hadoop.yarn.service.client.params.SliderActions; -import org.apache.slider.common.tools.ConfigHelper; -import org.apache.slider.common.tools.PortScanner; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.common.tools.SliderVersionInfo; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.exceptions.SliderInternalStateException; -import org.apache.slider.core.exceptions.TriggerClusterTeardownException; -import org.apache.slider.core.launch.CredentialUtils; -import org.apache.slider.core.main.ExitCodeProvider; -import org.apache.slider.core.main.LauncherExitCodes; -import org.apache.slider.core.main.RunService; -import org.apache.slider.core.main.ServiceLauncher; -import org.apache.slider.core.registry.info.CustomRegistryConstants; -import org.apache.slider.providers.ProviderCompleted; -import org.apache.slider.server.appmaster.actions.ActionHalt; -import org.apache.slider.server.appmaster.actions.ActionRegisterServiceInstance; -import org.apache.slider.server.appmaster.actions.ActionStopSlider; -import org.apache.slider.server.appmaster.actions.ActionUpgradeContainers; -import org.apache.slider.server.appmaster.actions.AsyncAction; -import org.apache.slider.server.appmaster.actions.EscalateOutstandingRequests; -import org.apache.slider.server.appmaster.actions.MonitorComponentInstances; -import org.apache.slider.server.appmaster.actions.QueueExecutor; -import org.apache.slider.server.appmaster.actions.QueueService; -import org.apache.slider.server.appmaster.actions.RegisterComponentInstance; -import org.apache.slider.server.appmaster.actions.RenewingAction; -import org.apache.slider.server.appmaster.actions.ResetFailureWindow; -import org.apache.slider.server.appmaster.actions.ReviewAndFlexApplicationSize; -import org.apache.slider.server.appmaster.actions.UnregisterComponentInstance; -import org.apache.slider.server.appmaster.management.MetricsAndMonitoring; -import org.apache.slider.server.appmaster.management.YarnServiceHealthCheck; -import org.apache.slider.server.appmaster.monkey.ChaosKillAM; -import org.apache.slider.server.appmaster.monkey.ChaosKillContainer; -import org.apache.slider.server.appmaster.monkey.ChaosMonkeyService; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.AsyncRMOperationHandler; -import org.apache.slider.server.appmaster.operations.RMOperationHandler; -import org.apache.slider.server.appmaster.rpc.RpcBinder; -import org.apache.slider.server.appmaster.rpc.SliderClusterProtocolPBImpl; -import org.apache.slider.server.appmaster.rpc.SliderIPCService; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.AppStateBindingInfo; -import org.apache.slider.server.appmaster.state.ContainerAssignment; -import org.apache.slider.server.appmaster.state.MostRecentContainerReleaseSelector; -import org.apache.slider.server.appmaster.state.ProviderAppState; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.hadoop.yarn.service.timelineservice.ServiceTimelinePublisher; -import org.apache.hadoop.yarn.service.timelineservice.ServiceMetricsSink; -import org.apache.slider.server.appmaster.web.SliderAMWebApp; -import org.apache.slider.server.appmaster.web.WebAppApi; -import org.apache.slider.server.appmaster.web.WebAppApiImpl; -import org.apache.slider.server.appmaster.web.rest.InsecureAmFilterInitializer; -import org.apache.slider.server.appmaster.web.rest.RestPaths; -import org.apache.slider.server.appmaster.web.rest.application.ApplicationResouceContentCacheFactory; -import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache; -import org.apache.slider.server.services.utility.AbstractSliderLaunchedService; -import org.apache.slider.server.services.utility.WebAppService; -import org.apache.slider.server.services.workflow.ServiceThreadFactory; -import org.apache.slider.server.services.workflow.WorkflowExecutorService; -import org.apache.slider.server.services.workflow.WorkflowRpcService; -import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders; -import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.net.URL; -import java.nio.ByteBuffer; -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Date; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.ReentrantLock; - -import static org.apache.hadoop.yarn.conf.YarnConfiguration.*; -import static org.apache.slider.common.Constants.HADOOP_JAAS_DEBUG; - -/** - * This is the AM, which directly implements the callbacks from the AM and NM - */ -public class SliderAppMaster extends AbstractSliderLaunchedService - implements AMRMClientAsync.CallbackHandler, - NMClientAsync.CallbackHandler, - RunService, - SliderExitCodes, - SliderKeys, - ServiceStateChangeListener, - RoleKeys, - ProviderCompleted, - AppMasterActionOperations { - - protected static final Logger log = - LoggerFactory.getLogger(SliderAppMaster.class); - - /** - * log for YARN events - */ - protected static final Logger LOG_YARN = log; - - public static final String SERVICE_CLASSNAME_SHORT = "SliderAppMaster"; - public static final String SERVICE_CLASSNAME = - "org.apache.slider.server.appmaster." + SERVICE_CLASSNAME_SHORT; - - public static final int HEARTBEAT_INTERVAL = 1000; - public static final int NUM_RPC_HANDLERS = 5; - - /** - * Metrics and monitoring services. - * Deployed in {@link #serviceInit(Configuration)} - */ - private final MetricsAndMonitoring metricsAndMonitoring = new MetricsAndMonitoring(); - - /** - * metrics registry - */ - public MetricRegistry metrics; - - /** Error string on chaos monkey launch failure action: {@value} */ - public static final String E_TRIGGERED_LAUNCH_FAILURE = - "Chaos monkey triggered launch failure"; - - /** YARN RPC to communicate with the Resource Manager or Node Manager */ - private YarnRPC yarnRPC; - - /** Handle to communicate with the Resource Manager*/ - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private AMRMClientAsync asyncRMClient; - - /** Handle to communicate with the timeline service */ - private TimelineV2Client timelineClient; - - private boolean timelineServiceEnabled = false; - - ServiceTimelinePublisher serviceTimelinePublisher; - - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private RMOperationHandler rmOperationHandler; - - /** Handle to communicate with the Node Manager*/ - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - public NMClientAsync nmClientAsync; - - /** - * Credentials for propagating down to launched containers - */ - private Credentials containerCredentials = new Credentials(); - - /** - * Slider IPC: Real service handler - */ - private SliderIPCService sliderIPCService; - /** - * Slider IPC: binding - */ - private WorkflowRpcService rpcService; - - /** - * Secret manager - */ - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private ClientToAMTokenSecretManager secretManager; - - /** Hostname of the container*/ - private String appMasterHostname = ""; - /* Port on which the app master listens for status updates from clients*/ - private int appMasterRpcPort = 0; - /** Tracking url to which app master publishes info for clients to monitor*/ - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private String appMasterTrackingUrl = ""; - - /** Proxied app master URL (as retrieved from AM report at launch time) */ - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private String appMasterProxiedUrl = ""; - - /** Application Attempt Id ( combination of attemptId and fail count )*/ - private ApplicationAttemptId appAttemptID; - - /** - * App ACLs - */ - protected Map applicationACLs; - - /** - * Ongoing state of the cluster: containers, nodes they - * live on, etc. - */ - private final AppState appState = - new AppState(new ProtobufClusterServices(), metricsAndMonitoring); - - /** - * App state for external objects. This is almost entirely - * a read-only view of the application state. To change the state, - * Providers (or anything else) are expected to queue async changes. - */ - private final ProviderAppState stateForProviders = - new ProviderAppState("undefined", appState); - - /** - * model the state using locks and conditions - */ - private final ReentrantLock AMExecutionStateLock = new ReentrantLock(); - private final Condition isAMCompleted = AMExecutionStateLock.newCondition(); - - /** - * Flag set if the AM is to be shutdown - */ - private final AtomicBoolean amCompletionFlag = new AtomicBoolean(false); - - /** - * Flag set during the init process - */ - private final AtomicBoolean initCompleted = new AtomicBoolean(false); - - /** Arguments passed in : raw*/ - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private SliderAMArgs serviceArgs; - - /** - * ID of the AM container - */ - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private ContainerId appMasterContainerID; - - /** - * Monkey Service -may be null - */ - private ChaosMonkeyService monkey; - - /** - * ProviderService of this cluster - */ - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private Set providers = new HashSet<>(); - - /** - * The YARN registry service - */ - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private RegistryOperations registryOperations; - - /** - * The stop request received...the exit details are extracted - * from this - */ - private volatile ActionStopSlider stopAction; - - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private RoleLaunchService launchService; - - //username -null if it is not known/not to be set - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private String hadoop_user_name; - private String service_user_name; - - private SliderAMWebApp webApp; - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private InetSocketAddress rpcServiceAddress; - - /** - * Executor. - * Assigned in {@link #serviceInit(Configuration)} - */ - private WorkflowExecutorService executorService; - - /** - * Action queues. Created at instance creation, but - * added as a child and inited in {@link #serviceInit(Configuration)} - */ - private final QueueService actionQueues = new QueueService(); - private YarnRegistryViewForProviders yarnRegistryOperations; - //private FsDelegationTokenManager fsDelegationTokenManager; - private RegisterApplicationMasterResponse amRegistrationData; - private PortScanner portScanner; - - /** - * Is security enabled? - * Set early on in the {@link #createAndRunCluster(String)} operation. - */ - private boolean securityEnabled; - private ContentCache contentCache; - - /** - * resource limits - */ - private Resource maximumResourceCapability; - private Application application; - /** - * Service Constructor - */ - public SliderAppMaster() { - super(SERVICE_CLASSNAME_SHORT); - new HdfsConfiguration(); - new YarnConfiguration(); - } - -/* =================================================================== */ -/* service lifecycle methods */ -/* =================================================================== */ - - @Override //AbstractService - public synchronized void serviceInit(Configuration conf) throws Exception { - // slider client if found - - Configuration customConf = SliderUtils.loadSliderClientXML(); - // Load in the server configuration - if it is actually on the Classpath - URL serverXmlUrl = ConfigHelper.getResourceUrl(SLIDER_SERVER_XML); - if (serverXmlUrl != null) { - log.info("Loading {} at {}", SLIDER_SERVER_XML, serverXmlUrl); - Configuration serverConf = ConfigHelper.loadFromResource(SLIDER_SERVER_XML); - ConfigHelper.mergeConfigurations(customConf, serverConf, - SLIDER_SERVER_XML, true); - } - serviceArgs.applyDefinitions(customConf); - serviceArgs.applyFileSystemBinding(customConf); - // conf now contains all customizations - - AbstractActionArgs action = serviceArgs.getCoreAction(); - SliderAMCreateAction createAction = (SliderAMCreateAction) action; - - // sort out the location of the AM - String rmAddress = createAction.getRmAddress(); - if (rmAddress != null) { - log.debug("Setting RM address from the command line: {}", rmAddress); - SliderUtils.setRmSchedulerAddress(customConf, rmAddress); - } - - log.info("AM configuration:\n{}", - ConfigHelper.dumpConfigToString(customConf)); - for (Map.Entry envs : System.getenv().entrySet()) { - log.info("System env {}={}", envs.getKey(), envs.getValue()); - } - - ConfigHelper.mergeConfigurations(conf, customConf, SLIDER_CLIENT_XML, true); - //init security with our conf - if (SliderUtils.isHadoopClusterSecure(conf)) { - log.info("Secure mode with kerberos realm {}", - SliderUtils.getKerberosRealm()); - UserGroupInformation.setConfiguration(conf); - UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); - log.debug("Authenticating as {}", ugi); - SliderUtils.verifyPrincipalSet(conf, DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY); - } else { - log.info("Cluster is insecure"); - } - log.info("Login user is {}", UserGroupInformation.getLoginUser()); - - //look at settings of Hadoop Auth, to pick up a problem seen once - checkAndWarnForAuthTokenProblems(); - - // validate server env - boolean dependencyChecks = - !conf.getBoolean(KEY_SLIDER_AM_DEPENDENCY_CHECKS_DISABLED, - false); - SliderUtils.validateSliderServerEnvironment(log, dependencyChecks); - - // create and register monitoring services - addService(metricsAndMonitoring); - metrics = metricsAndMonitoring.getMetrics(); -/* TODO: turn these one once the metrics testing is more under control - metrics.registerAll(new ThreadStatesGaugeSet()); - metrics.registerAll(new MemoryUsageGaugeSet()); - metrics.registerAll(new GarbageCollectorMetricSet()); - -*/ - contentCache = ApplicationResouceContentCacheFactory.createContentCache(stateForProviders); - - executorService = new WorkflowExecutorService<>("AmExecutor", - Executors.newFixedThreadPool(2, - new ServiceThreadFactory("AmExecutor", true))); - addService(executorService); - - addService(actionQueues); - if (YarnConfiguration.timelineServiceV2Enabled(conf)) { - timelineServiceEnabled = true; - log.info("Enabled YARN timeline service v2. "); - } - - //init all child services - super.serviceInit(conf); - } - - @Override - protected void serviceStart() throws Exception { - super.serviceStart(); - HealthCheckRegistry health = metricsAndMonitoring.getHealth(); - health.register("AM Health", new YarnServiceHealthCheck(this)); - } - - /** - * Start the queue processing - */ - private void startQueueProcessing() { - log.info("Queue Processing started"); - executorService.execute(actionQueues); - executorService.execute(new QueueExecutor(this, actionQueues)); - } - -/* =================================================================== */ -/* RunService methods called from ServiceLauncher */ -/* =================================================================== */ - - /** - * pick up the args from the service launcher - * @param config configuration - * @param args argument list - */ - @Override // RunService - public Configuration bindArgs(Configuration config, String... args) throws Exception { - // let the superclass process it - Configuration superConf = super.bindArgs(config, args); - - //yarn-ify - YarnConfiguration yarnConfiguration = new YarnConfiguration( - superConf); - serviceArgs = new SliderAMArgs(args); - serviceArgs.parse(); - - return SliderUtils.patchConfiguration(yarnConfiguration); - } - - - /** - * this is called by service launcher; when it returns the application finishes - * @return the exit code to return by the app - * @throws Throwable - */ - @Override - public int runService() throws Throwable { - SliderVersionInfo.loadAndPrintVersionInfo(log); - - //dump the system properties if in debug mode - if (log.isDebugEnabled()) { - log.debug("System properties:\n" + SliderUtils.propertiesToString(System.getProperties())); - } - - //choose the action - String action = serviceArgs.getAction(); - List actionArgs = serviceArgs.getActionArgs(); - int exitCode; - switch (action) { - case SliderActions.ACTION_HELP: - log.info("{}: {}", getName(), serviceArgs.usage()); - exitCode = SliderExitCodes.EXIT_USAGE; - break; - case SliderActions.ACTION_CREATE: - exitCode = createAndRunCluster(actionArgs.get(0)); - break; - default: - throw new SliderException("Unimplemented: " + action); - } - log.info("Exiting AM; final exit code = {}", exitCode); - return exitCode; - } - - /** - * Initialize a newly created service then add it. - * Because the service is not started, this MUST be done before - * the AM itself starts, or it is explicitly added after - * @param service the service to init - */ - public Service initAndAddService(Service service) { - service.init(getConfig()); - addService(service); - return service; - } - - /* =================================================================== */ - - /** - * Create and run the cluster. - * @param appName cluster name - * @return exit code - * @throws Throwable on a failure - */ - private int createAndRunCluster(String appName) throws Throwable { - Path appDir = new Path((serviceArgs.getAppDefPath())); - SliderFileSystem fs = getClusterFS(); - fs.setAppDir(appDir); - application = ServiceApiUtil.loadApplication(fs, appName); - log.info("Application Json: " + application); - stateForProviders.setApplicationName(appName); - Configuration serviceConf = getConfig(); - - // obtain security state - // set the global security flag for the instance definition - - // initialize our providers - for (Component component : application.getComponents()) { - ProviderFactory factory = ProviderFactory - .createSliderProviderFactory(component.getArtifact()); - ProviderService providerService = factory.createServerProvider(); - // init the provider BUT DO NOT START IT YET -// initAndAddService(providerService); - providers.add(providerService); - } - - InetSocketAddress rmSchedulerAddress = SliderUtils.getRmSchedulerAddress(serviceConf); - log.info("RM is at {}", rmSchedulerAddress); - yarnRPC = YarnRPC.create(serviceConf); - - // set up the YARN client. This may require patching in the RM client-API address if it - // is (somehow) unset server-side. String clientRMaddr = serviceConf.get(YarnConfiguration.RM_ADDRESS); - InetSocketAddress clientRpcAddress = SliderUtils.getRmAddress(serviceConf); - if (!SliderUtils.isAddressDefined(clientRpcAddress)) { - // client addr is being unset. We can lift it from the other RM APIs - log.warn("Yarn RM address was unbound; attempting to fix up"); - serviceConf.set(YarnConfiguration.RM_ADDRESS, - String.format("%s:%d", rmSchedulerAddress.getHostString(), clientRpcAddress.getPort() )); - } - - /* - * Extract the container ID. This is then - * turned into an (incomplete) container - */ - appMasterContainerID = ConverterUtils.toContainerId( - SliderUtils.mandatoryEnvVariable(ApplicationConstants.Environment.CONTAINER_ID.name())); - appAttemptID = appMasterContainerID.getApplicationAttemptId(); - - ApplicationId appid = appAttemptID.getApplicationId(); - log.info("AM for ID {}", appid.getId()); - - Map envVars; - List liveContainers; - - /* - * It is critical this section is synchronized, to stop async AM events - * arriving while registering a restarting AM. - */ - synchronized (appState) { - int heartbeatInterval = HEARTBEAT_INTERVAL; - - // configure AM to wait forever for RM - getConfig().setLong(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS, - -1); - getConfig().unset(YarnConfiguration.CLIENT_FAILOVER_MAX_ATTEMPTS); - - // add the RM client -this brings the callbacks in - asyncRMClient = AMRMClientAsync.createAMRMClientAsync(heartbeatInterval, this); - addService(asyncRMClient); - //now bring it up - deployChildService(asyncRMClient); - - if (timelineServiceEnabled) { - timelineClient = TimelineV2Client.createTimelineClient(appid); - asyncRMClient.registerTimelineV2Client(timelineClient); - timelineClient.init(getConfig()); - timelineClient.start(); - log.info("Timeline v2 client started."); - - serviceTimelinePublisher = new ServiceTimelinePublisher(timelineClient); - serviceTimelinePublisher.init(getConfig()); - serviceTimelinePublisher.start(); - - for (ProviderService providerService : providers) { - } - appState.setServiceTimelinePublisher(serviceTimelinePublisher); - log.info("ServiceTimelinePublisher started."); - } - - - // nmclient relays callbacks back to this class - nmClientAsync = new NMClientAsyncImpl("nmclient", this); - deployChildService(nmClientAsync); - - // set up secret manager - secretManager = new ClientToAMTokenSecretManager(appAttemptID, null); - - if (securityEnabled) { - // fix up the ACLs if they are not set - String acls = serviceConf.get(KEY_PROTOCOL_ACL); - if (acls == null) { - getConfig().set(KEY_PROTOCOL_ACL, "*"); - } - } - - //bring up the Slider RPC service - buildPortScanner(); - startSliderRPCServer(); - - rpcServiceAddress = rpcService.getConnectAddress(); - appMasterHostname = rpcServiceAddress.getAddress().getCanonicalHostName(); - appMasterRpcPort = rpcServiceAddress.getPort(); - appMasterTrackingUrl = null; - log.info("AM Server is listening at {}:{}", appMasterHostname, appMasterRpcPort); - - log.info("Starting Yarn registry"); - registryOperations = startRegistryOperationsService(); - log.info(registryOperations.toString()); - - // Start up the WebApp and track the URL for it - // Web service endpoints: initialize - WebAppApiImpl webAppApi = - new WebAppApiImpl( - stateForProviders, - registryOperations, - metricsAndMonitoring, - actionQueues); - initAMFilterOptions(serviceConf); - - int webAppPort = deployWebApplication(webAppApi); - - String scheme = WebAppUtils.HTTP_PREFIX; - appMasterTrackingUrl = scheme + appMasterHostname + ":" + webAppPort; - - // ***************************************************** - // Register self with ResourceManager - // This will start heartbeating to the RM - // address = SliderUtils.getRmSchedulerAddress(asyncRMClient.getConfig()); - // ***************************************************** - log.info("Connecting to RM at {}; AM tracking URL={}", - appMasterRpcPort, appMasterTrackingUrl); - amRegistrationData = asyncRMClient.registerApplicationMaster(appMasterHostname, - appMasterRpcPort, - appMasterTrackingUrl); - maximumResourceCapability = amRegistrationData.getMaximumResourceCapability(); - - //TODO should not read local configs !!! - int minMemory = serviceConf.getInt(RM_SCHEDULER_MINIMUM_ALLOCATION_MB, - DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB); - // validate scheduler vcores allocation setting - int minCores = serviceConf.getInt(RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, - DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); - int maxMemory = maximumResourceCapability.getMemory(); - int maxCores = maximumResourceCapability.getVirtualCores(); - appState.setContainerLimits(minMemory,maxMemory, minCores, maxCores ); - - // build the handler for RM request/release operations; this uses - // the max value as part of its lookup - rmOperationHandler = new AsyncRMOperationHandler(asyncRMClient, maximumResourceCapability); - - stripAMRMToken(); - -// if (securityEnabled) { -// secretManager.setMasterKey( -// amRegistrationData.getClientToAMTokenMasterKey().array()); -// applicationACLs = amRegistrationData.getApplicationACLs(); -// -// //tell the server what the ACLs are -// rpcService.getServer().refreshServiceAcl(serviceConf, -// new SliderAMPolicyProvider()); -// if (securityConfiguration.isKeytabProvided()) { -// // perform keytab based login to establish kerberos authenticated -// // principal. Can do so now since AM registration with RM above required -// // tokens associated to principal -// String principal = securityConfiguration.getPrincipal(); -// //TODO read key tab file from slider-am.xml -// File localKeytabFile = new File("todo"); -//// securityConfiguration.getKeytabFile(new AggregateConf()); -// // Now log in... -// login(principal, localKeytabFile); -// // obtain new FS reference that should be kerberos based and different -// // than the previously cached reference -// fs = new SliderFileSystem(serviceConf); -// } -// } - - // YARN client. - // Important: this is only valid at startup, and must be executed within - // the right UGI context. Use with care. - YarnClient yarnClient = null; - List nodeReports; - try { - yarnClient = YarnClient.createYarnClient(); - yarnClient.init(getConfig()); - yarnClient.start(); - nodeReports = getNodeReports(yarnClient); - log.info("Yarn node report count: {}", nodeReports.size()); - // look up the application itself -this is needed to get the proxied - // URL of the AM, for registering endpoints. - // this call must be made after the AM has registered itself, obviously - ApplicationAttemptReport report = getApplicationAttemptReport(yarnClient); - appMasterProxiedUrl = report.getTrackingUrl(); - if (SliderUtils.isUnset(appMasterProxiedUrl)) { - log.warn("Proxied URL is not set in application report"); - appMasterProxiedUrl = appMasterTrackingUrl; - } - } finally { - // at this point yarnClient is no longer needed. - // stop it immediately - ServiceOperations.stop(yarnClient); - yarnClient = null; - } - - // extract container list - - liveContainers = amRegistrationData.getContainersFromPreviousAttempts(); - DefaultMetricsSystem.initialize("SliderAppMaster"); - if (timelineServiceEnabled) { - DefaultMetricsSystem.instance().register("ServiceMetricsSink", - "For processing metrics to ATS", - new ServiceMetricsSink(serviceTimelinePublisher)); - log.info("ServiceMetricsSink registered."); - } - - //determine the location for the role history data - Path historyDir = new Path(appDir, HISTORY_DIR_NAME); - - //build the instance - AppStateBindingInfo binding = new AppStateBindingInfo(); - binding.serviceConfig = null; - binding.fs = fs.getFileSystem(); - binding.historyPath = historyDir; - binding.liveContainers = liveContainers; - binding.releaseSelector = new MostRecentContainerReleaseSelector(); - binding.nodeReports = nodeReports; - binding.application = application; - binding.serviceHdfsDir = new Path(fs.buildClusterDirPath(appName), - SliderKeys.DATA_DIR_NAME).toString(); - appState.buildInstance(binding); - - // build up environment variables that the AM wants set in every container - // irrespective of provider and role. - envVars = new HashMap<>(); - if (hadoop_user_name != null) { - envVars.put(HADOOP_USER_NAME, hadoop_user_name); - } - String debug_kerberos = System.getenv(HADOOP_JAAS_DEBUG); - if (debug_kerberos != null) { - envVars.put(HADOOP_JAAS_DEBUG, debug_kerberos); - } - } - String rolesTmpSubdir = appMasterContainerID.toString() + "/roles"; - - String amTmpDir = "/tmp"; - //TODO read tmpDir from slider-am.xml - Path tmpDirPath = new Path(amTmpDir); - Path launcherTmpDirPath = new Path(tmpDirPath, rolesTmpSubdir); - fs.getFileSystem().mkdirs(launcherTmpDirPath); - - //launcher service - launchService = new RoleLaunchService(actionQueues, - fs, envVars); - - deployChildService(launchService); - - //Give the provider access to the state, and AM - for (ProviderService providerService : providers) { -// providerService.setAMState(stateForProviders); - } - - // chaos monkey - maybeStartMonkey(); - - // if not a secure cluster, extract the username -it will be - // propagated to workers - if (!UserGroupInformation.isSecurityEnabled()) { - hadoop_user_name = System.getenv(HADOOP_USER_NAME); - log.info(HADOOP_USER_NAME + "='{}'", hadoop_user_name); - } - service_user_name = RegistryUtils.currentUser(); - log.info("Registry service username ={}", service_user_name); - - - // declare the cluster initialized - log.info("Application Master Initialization Completed"); - initCompleted.set(true); - - scheduleFailureWindowResets(application.getConfiguration()); - scheduleEscalation(application.getConfiguration()); - scheduleMonitoring(application.getConfiguration()); - - try { - // schedule YARN Registry registration - queue(new ActionRegisterServiceInstance(appName, appid, application)); - - // log the YARN and web UIs - log.info("RM Webapp address {}", - serviceConf.get(YarnConfiguration.RM_WEBAPP_ADDRESS)); - log.info("Slider webapp address {} proxied at {}", - appMasterTrackingUrl, appMasterProxiedUrl); - // launch the real provider; this is expected to trigger a callback that - // starts the node review process - launchProviderService(); - - // start handling any scheduled events - - startQueueProcessing(); - - //now block waiting to be told to exit the process - waitForAMCompletionSignal(); - } catch(Exception e) { - log.error("Exception : {}", e, e); - // call the AM stop command as if it had been queued (but without - // going via the queue, which may not have started - onAMStop(new ActionStopSlider(e)); - } - //shutdown time - return finish(); - } - - /** - * Get the YARN application Attempt report as the logged in user - * @param yarnClient client to the RM - * @return the application report - * @throws YarnException - * @throws IOException - * @throws InterruptedException - */ - private ApplicationAttemptReport getApplicationAttemptReport( - final YarnClient yarnClient) - throws YarnException, IOException, InterruptedException { - Preconditions.checkNotNull(yarnClient, "Null Yarn client"); - ApplicationAttemptReport report; - if (securityEnabled) { - UserGroupInformation ugi = UserGroupInformation.getLoginUser(); - report = ugi.doAs(new PrivilegedExceptionAction() { - @Override - public ApplicationAttemptReport run() throws Exception { - return yarnClient.getApplicationAttemptReport(appAttemptID); - } - }); - } else { - report = yarnClient.getApplicationAttemptReport(appAttemptID); - } - return report; - } - - /** - * List the node reports: uses {@link YarnClient} as the login user - * @param yarnClient client to the RM - * @return the node reports - * @throws IOException - * @throws YarnException - * @throws InterruptedException - */ - private List getNodeReports(final YarnClient yarnClient) - throws IOException, YarnException, InterruptedException { - Preconditions.checkNotNull(yarnClient, "Null Yarn client"); - List nodeReports; - if (securityEnabled) { - nodeReports = UserGroupInformation.getLoginUser().doAs( - new PrivilegedExceptionAction>() { - @Override - public List run() throws Exception { - return yarnClient.getNodeReports(NodeState.RUNNING); - } - }); - } else { - nodeReports = yarnClient.getNodeReports(NodeState.RUNNING); - } - log.info("Yarn node report count: {}", nodeReports.size()); - return nodeReports; - } - - /** - * Deploy the web application. - *

- * Creates and starts the web application, and adds a - * WebAppService service under the AM, to ensure - * a managed web application shutdown. - * @param webAppApi web application API instance - * @return port the web application is deployed on - * @throws IOException general problems starting the webapp (network, etc) - * @throws WebAppException other issues - */ - private int deployWebApplication(WebAppApiImpl webAppApi) - throws IOException, SliderException { - - try { - webApp = new SliderAMWebApp(webAppApi); - HttpConfig.Policy policy = HttpConfig.Policy.HTTP_ONLY; - int port = getPortToRequest(); - log.info("Launching web application at port {} with policy {}", port, policy); - - WebApps.$for(SliderAMWebApp.BASE_PATH, - WebAppApi.class, - webAppApi, - RestPaths.WS_CONTEXT) - .withHttpPolicy(getConfig(), policy) - .at("0.0.0.0", port, true) - .inDevMode() - .start(webApp); - - WebAppService webAppService = - new WebAppService<>("slider", webApp); - - deployChildService(webAppService); - return webApp.port(); - } catch (WebAppException e) { - if (e.getCause() instanceof IOException) { - throw (IOException)e.getCause(); - } else { - throw e; - } - } - } - - /** - * Process the initial user to obtain the set of user - * supplied credentials (tokens were passed in by client). - * Removes the AM/RM token. - * @throws IOException - */ - private void stripAMRMToken() - throws IOException { - List filteredTokens = new ArrayList<>(3); - filteredTokens.add(AMRMTokenIdentifier.KIND_NAME); - containerCredentials = CredentialUtils.filterTokens( - UserGroupInformation.getCurrentUser().getCredentials(), - filteredTokens); - log.info(CredentialUtils.dumpTokens(containerCredentials, "\n")); - } - - /** - * Build up the port scanner. This may include setting a port range. - */ - private void buildPortScanner() - throws BadConfigException { - portScanner = new PortScanner(); - String portRange = "0"; - //TODO read from slider-am.xml -// String portRange = instanceDefinition. -// getAppConfOperations().getGlobalOptions(). -// getOption(SliderKeys.KEY_ALLOWED_PORT_RANGE, "0"); - if (!"0".equals(portRange)) { - portScanner.setPortRange(portRange); - } - } - - /** - * Locate a port to request for a service such as RPC or web/REST. - * This uses port range definitions in the instanceDefinition - * to fix the port range —if one is set. - *

- * The port returned is available at the time of the request; there are - * no guarantees as to how long that situation will last. - * @return the port to request. - * @throws SliderException - */ - private int getPortToRequest() throws SliderException, IOException { - return portScanner.getAvailablePort(); - } - - protected void login(String principal, File localKeytabFile) - throws IOException, SliderException { - log.info("Logging in as {} with keytab {}", principal, localKeytabFile); - UserGroupInformation.loginUserFromKeytab(principal, - localKeytabFile.getAbsolutePath()); - validateLoginUser(UserGroupInformation.getLoginUser()); - } - - /** - * Ensure that the user is generated from a keytab and has no HDFS delegation - * tokens. - * - * @param user user to validate - * @throws SliderException - */ - protected void validateLoginUser(UserGroupInformation user) - throws SliderException { - if (!user.isFromKeytab()) { - log.error("User is not holding on a keytab in a secure deployment:" + - " slider will fail as tokens expire"); - } - Credentials credentials = user.getCredentials(); - Iterator> iter = - credentials.getAllTokens().iterator(); - while (iter.hasNext()) { - Token token = iter.next(); - log.info("Token {}", token.getKind()); - if (token.getKind().equals( - DelegationTokenIdentifier.HDFS_DELEGATION_KIND)) { - log.info("HDFS delegation token {}. Removing...", token); - iter.remove(); - } - } - } - - /** - * Set up the AM filter - * @param serviceConf configuration to patch - */ - private void initAMFilterOptions(Configuration serviceConf) { - // IP filtering - String amFilterName = AM_FILTER_NAME; - - // This is here until YARN supports proxy & redirect operations - // on verbs other than GET, and is only supported for testing - if (X_DEV_INSECURE_REQUIRED && serviceConf.getBoolean(X_DEV_INSECURE_WS, - X_DEV_INSECURE_DEFAULT)) { - log.warn("Insecure filter enabled: REST operations are unauthenticated"); - amFilterName = InsecureAmFilterInitializer.NAME; - } - - serviceConf.set(HADOOP_HTTP_FILTER_INITIALIZERS, amFilterName); - } - - /** - * This registers the service instance and its external values - * @param instanceName name of this instance - * @param appId application ID - * @throws IOException - */ - public void registerServiceInstance(String instanceName, - ApplicationId appId, Application application) throws IOException { - - //Give the provider restricted access to the state, registry - setupInitialRegistryPaths(); - yarnRegistryOperations = new YarnRegistryViewForProviders( - registryOperations, - service_user_name, - SliderKeys.APP_TYPE, - instanceName, - appAttemptID); - for (ProviderService providerService : providers) { -// providerService.bindToYarnRegistry(yarnRegistryOperations); - } - - // Yarn registry - ServiceRecord serviceRecord = new ServiceRecord(); - serviceRecord.set(YarnRegistryAttributes.YARN_ID, appId.toString()); - serviceRecord.set(YarnRegistryAttributes.YARN_PERSISTENCE, - PersistencePolicies.APPLICATION); - serviceRecord.description = "Slider Application Master"; - - serviceRecord.addExternalEndpoint( - RegistryTypeUtils.ipcEndpoint( - CustomRegistryConstants.AM_IPC_PROTOCOL, - rpcServiceAddress)); - - // set any provided attributes - setUserProvidedServiceRecordAttributes(application.getConfiguration(), - serviceRecord); - - // register the service's entry - log.info("Service Record \n{}", serviceRecord); - yarnRegistryOperations.registerSelf(serviceRecord, true); - log.info("Registered service under {}; absolute path {}", - yarnRegistryOperations.getSelfRegistrationPath(), - yarnRegistryOperations.getAbsoluteSelfRegistrationPath()); - - boolean isFirstAttempt = 1 == appAttemptID.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); - } - if (timelineServiceEnabled) { - serviceTimelinePublisher.serviceAttemptRegistered(application); - } - } - - /** - * TODO: purge this once RM is doing the work - * @throws IOException - */ - protected void setupInitialRegistryPaths() throws IOException { - if (registryOperations instanceof RMRegistryOperationsService) { - RMRegistryOperationsService rmRegOperations = - (RMRegistryOperationsService) registryOperations; - rmRegOperations.initUserRegistryAsync(service_user_name); - } - } - - /** - * Handler for {@link RegisterComponentInstance action} - * Register/re-register an ephemeral container that is already in the application state - * @param id the component - * @return true if the component is registered - */ - public boolean registerComponent(ContainerId id, RoleInstance roleInstance) - throws IOException { - RoleInstance instance = appState.getOwnedContainer(id); - if (instance == null) { - return false; - } - // this is where component registrations go - log.info("Registering component " + roleInstance.getCompInstanceName() - + ", containerId = " + id); - org.apache.slider.api.resource.Container container = - new org.apache.slider.api.resource.Container(); - container.setId(id.toString()); - container.setLaunchTime(new Date()); - container.setState(org.apache.slider.api.resource.ContainerState.RUNNING_BUT_UNREADY); - container.setBareHost(instance.host); - // TODO differentiate component name and component instance name ? - container.setComponentName(roleInstance.getCompInstanceName()); - instance.providerRole.component.addContainer(container); - - if (timelineServiceEnabled) { - serviceTimelinePublisher.componentInstanceStarted(container, null); - } - return true; - } - - protected void setUserProvidedServiceRecordAttributes( - org.apache.slider.api.resource.Configuration conf, ServiceRecord record) { - String prefix = RoleKeys.SERVICE_RECORD_ATTRIBUTE_PREFIX; - 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()); - } - } - } - - /** - * Handler for {@link UnregisterComponentInstance} - * - * unregister a component. At the time this message is received, - * the component may not have been registered - */ - public void unregisterComponent(RoleInstance roleInstance) { - ContainerId containerId = roleInstance.getContainerId(); - log.info( - "Unregistering component instance " + roleInstance.getCompInstanceName() - + ", ContainerId = " + containerId); - if (yarnRegistryOperations == null) { - log.warn("Processing unregister component event before initialization " - + "completed; init flag ={}", initCompleted); - return; - } - String cid = RegistryPathUtils.encodeYarnID(containerId.toString()); -// try { -// yarnRegistryOperations.deleteComponent(cid); -// } catch (IOException e) { -// log.warn("Failed to delete container {} : {}", containerId, e, e); -// } - - // remove component instance dir - try { - FileSystem fs = getClusterFS().getFileSystem(); - if (roleInstance.compInstanceDir != null && fs - .exists(roleInstance.compInstanceDir)) { - boolean deleted = fs.delete(roleInstance.compInstanceDir, true); - if (!deleted) { - log.warn("Failed to delete component instance dir: " - + roleInstance.compInstanceDir); - } - } - } catch (IOException e) { - log.error("Failed to delete component instance dir: " - + roleInstance.compInstanceDir, e); - } - } - - /** - * looks for a specific case where a token file is provided as an environment - * variable, yet the file is not there. - * - * This surfaced (once) in HBase, where its HDFS library was looking for this, - * and somehow the token was missing. This is a check in the AM so that - * if the problem re-occurs, the AM can fail with a more meaningful message. - * - */ - private void checkAndWarnForAuthTokenProblems() { - String fileLocation = - System.getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION); - if (fileLocation != null) { - File tokenFile = new File(fileLocation); - if (!tokenFile.exists()) { - log.warn("Token file {} specified in {} not found", tokenFile, - UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION); - } - } - } - - /** - * Get the filesystem of this cluster - * @return the FS of the config - */ - public SliderFileSystem getClusterFS() throws IOException { - return new SliderFileSystem(getConfig()); - } - - /** - * Get the AM log - * @return the log of the AM - */ - public static Logger getLog() { - return log; - } - - /** - * Get the application state - * @return the application state - */ - public AppState getAppState() { - return appState; - } - - /** - * Block until it is signalled that the AM is done - */ - private void waitForAMCompletionSignal() { - AMExecutionStateLock.lock(); - try { - if (!amCompletionFlag.get()) { - log.debug("blocking until signalled to terminate"); - isAMCompleted.awaitUninterruptibly(); - } - } finally { - AMExecutionStateLock.unlock(); - } - } - - /** - * Signal that the AM is complete .. queues it in a separate thread - * - * @param stopActionRequest request containing shutdown details - */ - public synchronized void signalAMComplete(ActionStopSlider stopActionRequest) { - // this is a queued action: schedule it through the queues - schedule(stopActionRequest); - } - - /** - * Signal that the AM is complete - * - * @param stopActionRequest request containing shutdown details - */ - public synchronized void onAMStop(ActionStopSlider stopActionRequest) { - - AMExecutionStateLock.lock(); - try { - if (amCompletionFlag.compareAndSet(false, true)) { - // first stop request received - this.stopAction = stopActionRequest; - isAMCompleted.signal(); - } - } finally { - AMExecutionStateLock.unlock(); - } - } - - - /** - * trigger the YARN cluster termination process - * @return the exit code - * @throws Exception if the stop action contained an Exception which implements - * ExitCodeProvider - */ - private synchronized int finish() throws Exception { - Preconditions.checkNotNull(stopAction, "null stop action"); - FinalApplicationStatus appStatus; - log.info("Triggering shutdown of the AM: {}", stopAction); - - String appMessage = stopAction.getMessage(); - //stop the daemon & grab its exit code - int exitCode = stopAction.getExitCode(); - Exception exception = stopAction.getEx(); - - appStatus = stopAction.getFinalApplicationStatus(); - - // make sure the AM is actually registered. If not, there's no point - // trying to unregister it - if (amRegistrationData == null) { - log.info("Application attempt not yet registered; skipping unregistration"); - if (exception != null) { - throw exception; - } - return exitCode; - } - - //stop any launches in progress - launchService.stop(); - - //now release all containers - releaseAllContainers(application); - DefaultMetricsSystem.shutdown(); - - if (timelineServiceEnabled) { - serviceTimelinePublisher.serviceAttemptUnregistered(appState, stopAction); - serviceTimelinePublisher.stop(); - timelineClient.stop(); - } - - // When the application completes, it should send a finish application - // signal to the RM - log.info("Application completed. Signalling finish to RM"); - - try { - log.info("Unregistering AM status={} message={}", appStatus, appMessage); - asyncRMClient.unregisterApplicationMaster(appStatus, appMessage, null); - } catch (InvalidApplicationMasterRequestException e) { - log.info("Application not found in YARN application list;" + - " it may have been terminated/YARN shutdown in progress: {}", e, e); - } catch (YarnException | IOException e) { - log.info("Failed to unregister application: " + e, e); - } - if (exception != null) { - throw exception; - } - return exitCode; - } - - - public Object getProxy(Class protocol, InetSocketAddress addr) { - return yarnRPC.getProxy(protocol, addr, getConfig()); - } - - /** - * Start the slider RPC server - */ - private void startSliderRPCServer() - throws IOException, SliderException { - verifyIPCAccess(); - - sliderIPCService = new SliderIPCService( - this, stateForProviders, - actionQueues, - metricsAndMonitoring, - contentCache); - - deployChildService(sliderIPCService); - SliderClusterProtocolPBImpl protobufRelay = - new SliderClusterProtocolPBImpl(sliderIPCService); - BlockingService blockingService = SliderClusterAPI.SliderClusterProtocolPB - .newReflectiveBlockingService( - protobufRelay); - - int port = getPortToRequest(); - InetSocketAddress rpcAddress = new InetSocketAddress("0.0.0.0", port); - rpcService = - new WorkflowRpcService("SliderRPC", - RpcBinder.createProtobufServer(rpcAddress, getConfig(), - secretManager, - NUM_RPC_HANDLERS, - blockingService, - null)); - deployChildService(rpcService); - } - - /** - * verify that if the cluster is authed, the ACLs are set. - * @throws BadConfigException if Authorization is set without any ACL - */ - private void verifyIPCAccess() throws BadConfigException { - boolean authorization = getConfig().getBoolean( - CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, - false); - String acls = getConfig().get(KEY_PROTOCOL_ACL); - if (authorization && SliderUtils.isUnset(acls)) { - throw new BadConfigException("Application has IPC authorization enabled in " + - CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION + - " but no ACLs in " + KEY_PROTOCOL_ACL); - } - } - - -/* =================================================================== */ -/* AMRMClientAsync callbacks */ -/* =================================================================== */ - - /** - * Callback event when a container is allocated. - * - * The app state is updated with the allocation, and builds up a list - * of assignments and RM operations. The assignments are - * handed off into the pool of service launchers to asynchronously schedule - * container launch operations. - * - * The operations are run in sequence; they are expected to be 0 or more - * release operations (to handle over-allocations) - * - * @param allocatedContainers list of containers that are now ready to be - * given work. - */ - @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") - @Override //AMRMClientAsync - public void onContainersAllocated(List allocatedContainers) { - LOG_YARN.info("onContainersAllocated({})", allocatedContainers.size()); - List assignments = new ArrayList<>(); - List operations = new ArrayList<>(); - - //app state makes all the decisions - appState.onContainersAllocated(allocatedContainers, assignments, operations); - - //for each assignment: instantiate that role - for (ContainerAssignment assignment : assignments) { - //TODO Do we need to pass credentials to containers? - launchService.launchRole(assignment, application, null); - } - - //for all the operations, exec them - execute(operations); - } - - @Override //AMRMClientAsync - public synchronized void onContainersCompleted(List completedContainers) { - LOG_YARN.info("onContainersCompleted([{}]", completedContainers.size()); - for (ContainerStatus status : completedContainers) { - ContainerId containerId = status.getContainerId(); - LOG_YARN.info("Container Completion for" + - " containerID={}," + - " state={}," + - " exitStatus={}," + - " diagnostics={}", - containerId, status.getState(), - status.getExitStatus(), - status.getDiagnostics()); - - // non complete containers should not be here - assert (status.getState() == ContainerState.COMPLETE); - AppState.NodeCompletionResult result = appState.onCompletedContainer(status); - if (result.containerFailed) { - RoleInstance ri = result.roleInstance; - log.error("Role instance {} failed ", ri); - } - - // known nodes trigger notifications - if(!result.unknownNode) { - queue(new UnregisterComponentInstance(0, - TimeUnit.MILLISECONDS, result.roleInstance)); - - if (timelineServiceEnabled && result.roleInstance != null) { - serviceTimelinePublisher - .componentInstanceFinished(result.roleInstance); - } - } - } - - reviewRequestAndReleaseNodes("onContainersCompleted"); - } - - /** - * Signal that containers are being upgraded. Containers specified with - * --containers option and all containers of all roles specified with - * --components option are merged and upgraded. - * - * @param upgradeContainersRequest - * request containing upgrade details - */ - public synchronized void onUpgradeContainers( - ActionUpgradeContainers upgradeContainersRequest) throws IOException, - SliderException { - LOG_YARN.info("onUpgradeContainers({})", - upgradeContainersRequest.getMessage()); - Set containers = upgradeContainersRequest.getContainers() == null ? new HashSet() - : upgradeContainersRequest.getContainers(); - LOG_YARN.info(" Container list provided (total {}) : {}", - containers.size(), containers); - Set components = upgradeContainersRequest.getComponents() == null ? new HashSet() - : upgradeContainersRequest.getComponents(); - LOG_YARN.info(" Component list provided (total {}) : {}", - components.size(), components); - // If components are specified as well, then grab all the containers of - // each of the components (roles) - if (CollectionUtils.isNotEmpty(components)) { - Map liveContainers = appState.getLiveContainers(); - if (CollectionUtils.isNotEmpty(liveContainers.keySet())) { - Map> roleContainerMap = prepareRoleContainerMap(liveContainers); - for (String component : components) { - Set roleContainers = roleContainerMap.get(component); - if (roleContainers != null) { - containers.addAll(roleContainers); - } - } - } - } - LOG_YARN.info("Final list of containers to be upgraded (total {}) : {}", - containers.size(), containers); - } - - // create a reverse map of roles -> set of all live containers - private Map> prepareRoleContainerMap( - Map liveContainers) { - // liveContainers is ensured to be not empty - Map> roleContainerMap = new HashMap<>(); - for (Map.Entry liveContainer : liveContainers - .entrySet()) { - RoleInstance role = liveContainer.getValue(); - if (roleContainerMap.containsKey(role.role)) { - roleContainerMap.get(role.role).add(liveContainer.getKey().toString()); - } else { - Set containers = new HashSet(); - containers.add(liveContainer.getKey().toString()); - roleContainerMap.put(role.role, containers); - } - } - return roleContainerMap; - } - - /** - * Implementation of cluster flexing. - * It should be the only way that anything -even the AM itself on startup- - * asks for nodes. - * @throws SliderException slider problems, including invalid configs - * @throws IOException IO problems - */ - public void flexCluster(Messages.FlexComponentsRequestProto request) - throws IOException, SliderException { - if (request != null) { - appState.updateComponents(request); - } - // reset the scheduled windows...the values - // may have changed - appState.resetFailureCounts(); - - // ask for more containers if needed - reviewRequestAndReleaseNodes("flexCluster"); - } - - /** - * Schedule the failure window - * @throws BadConfigException if the window is out of range - */ - private void scheduleFailureWindowResets( - org.apache.slider.api.resource.Configuration conf) { - - ResetFailureWindow reset = new ResetFailureWindow(rmOperationHandler); - - long totalSeconds = SliderUtils.getTimeRange(conf, - ResourceKeys.CONTAINER_FAILURE_WINDOW, - ResourceKeys.DEFAULT_CONTAINER_FAILURE_WINDOW_DAYS, - ResourceKeys.DEFAULT_CONTAINER_FAILURE_WINDOW_HOURS, - ResourceKeys.DEFAULT_CONTAINER_FAILURE_WINDOW_MINUTES, - 0); - if (totalSeconds > 0) { - log.info("Scheduling the failure window reset interval to every {}" - + " seconds", totalSeconds); - RenewingAction renew = - new RenewingAction<>(reset, totalSeconds, totalSeconds, - TimeUnit.SECONDS, 0); - actionQueues.renewing("failures", renew); - } else { - log.info("Failure window reset interval is not set"); - } - } - - /** - * Schedule the escalation action - * @throws BadConfigException - */ - private void scheduleEscalation( - org.apache.slider.api.resource.Configuration conf) { - EscalateOutstandingRequests escalate = new EscalateOutstandingRequests(); - long seconds = conf.getPropertyLong(InternalKeys.ESCALATION_CHECK_INTERVAL, - InternalKeys.DEFAULT_ESCALATION_CHECK_INTERVAL); - RenewingAction renew = - new RenewingAction<>(escalate, seconds, seconds, TimeUnit.SECONDS, 0); - actionQueues.renewing("escalation", renew); - } - - /** - * Schedule monitor action - */ - private void scheduleMonitoring( - org.apache.slider.api.resource.Configuration conf) { - MonitorComponentInstances monitor = new MonitorComponentInstances(); - long seconds = conf.getPropertyLong(InternalKeys.MONITOR_INTERVAL, - InternalKeys.DEFAULT_MONITOR_INTERVAL); - RenewingAction renew = - new RenewingAction<>(monitor, seconds, seconds, TimeUnit.SECONDS, 0); - actionQueues.renewing("monitoring", renew); - } - - /** - * Look at where the current node state is and whether it should be changed. - * @param reason reason for operation - */ - private synchronized void reviewRequestAndReleaseNodes(String reason) { - log.info("reviewRequestAndReleaseNodes({})", reason); - queue(new ReviewAndFlexApplicationSize(reason, 0, TimeUnit.SECONDS)); - } - - /** - * Handle the event requesting a review ... look at the queue and decide - * whether to act or not - * @param action action triggering the event. It may be put - * back into the queue - * @throws SliderInternalStateException - */ - public void handleReviewAndFlexApplicationSize(ReviewAndFlexApplicationSize action) - throws SliderInternalStateException { - - if ( actionQueues.hasQueuedActionWithAttribute( - AsyncAction.ATTR_REVIEWS_APP_SIZE | AsyncAction.ATTR_HALTS_APP)) { - //TODO Loop all actions to check duplicate ?? - // this operation isn't needed at all -existing duplicate or shutdown due - return; - } - // if there is an action which changes cluster size, wait - if (actionQueues.hasQueuedActionWithAttribute( - AsyncAction.ATTR_CHANGES_APP_SIZE)) { - // place the action at the back of the queue - actionQueues.put(action); - } - - executeNodeReview(action.name); - } - - /** - * Look at where the current node state is -and whether it should be changed - */ - public synchronized void executeNodeReview(String reason) - throws SliderInternalStateException { - - log.info("in executeNodeReview({})", reason); - if (amCompletionFlag.get()) { - log.info("Ignoring node review operation: shutdown in progress"); - } - try { - List allOperations = appState.reviewRequestAndReleaseNodes(); - //now apply the operations - execute(allOperations); - } catch (TriggerClusterTeardownException e) { - //App state has decided that it is time to exit - log.error("Cluster teardown triggered {}", e, e); - queue(new ActionStopSlider(e)); - } - } - - /** - * Escalate operation as triggered by external timer. - *

- * Get the list of new operations off the AM, then executest them. - */ - public void escalateOutstandingRequests() { - List operations = appState.escalateOutstandingRequests(); - execute(operations); - } - - public void monitorComponentInstances() { - // TODO use health checks? - // TODO publish timeline events for monitoring changes? - if (appState.monitorComponentInstances()) { - // monitoring change - reviewRequestAndReleaseNodes("monitoring change"); - } - } - - - /** - * Shutdown operation: release all containers - */ - private void releaseAllContainers(Application application) { - // Add the sleep here (before releasing containers) so that applications get - // time to perform graceful shutdown - try { - long timeout = getContainerReleaseTimeout(application); - if (timeout > 0) { - Thread.sleep(timeout); - } - } catch (InterruptedException e) { - log.info("Sleep for container release interrupted"); - } finally { - List operations = appState.releaseAllContainers(); - // now apply the operations - execute(operations); - } - } - - private long getContainerReleaseTimeout(Application application) { - // Get container release timeout in millis or 0 if the property is not set. - long timeout = application.getConfiguration() - .getPropertyLong(SliderKeys.APP_CONTAINER_RELEASE_TIMEOUT, 0); - - // convert to millis - long timeoutInMillis = timeout * 1000l; - log.info("Container release timeout in millis = {}", timeoutInMillis); - return timeoutInMillis; - } - - /** - * RM wants to shut down the AM - */ - @Override //AMRMClientAsync - public void onShutdownRequest() { - LOG_YARN.info("Shutdown Request received"); - signalAMComplete(new ActionStopSlider("stop", - EXIT_SUCCESS, - FinalApplicationStatus.SUCCEEDED, - "Shutdown requested from RM")); - } - - /** - * Monitored nodes have been changed - * @param updatedNodes list of updated nodes - */ - @Override //AMRMClientAsync - public void onNodesUpdated(List updatedNodes) { - LOG_YARN.info("onNodesUpdated({})", updatedNodes.size()); - log.info("Updated nodes {}", updatedNodes); - // Check if any nodes are lost or revived and update state accordingly - - AppState.NodeUpdatedOutcome outcome = appState.onNodesUpdated(updatedNodes); - if (!outcome.operations.isEmpty()) { - execute(outcome.operations); - } - // trigger a review if the cluster changed - if (outcome.clusterChanged) { - reviewRequestAndReleaseNodes("nodes updated"); - } - } - - /** - * heartbeat operation; return the ratio of requested - * to actual - * @return progress - */ - @Override //AMRMClientAsync - public float getProgress() { - return appState.getApplicationProgressPercentage(); - } - - @Override //AMRMClientAsync - public void onError(Throwable e) { - if (e instanceof InvalidResourceRequestException) { - // stop the cluster - LOG_YARN.error("AMRMClientAsync.onError() received {}", e, e); - signalAMComplete(new ActionStopSlider("stop", EXIT_EXCEPTION_THROWN, - FinalApplicationStatus.FAILED, - SliderUtils.extractFirstLine(e.getLocalizedMessage()))); - } else if (e instanceof InvalidApplicationMasterRequestException) { - // halt the AM - LOG_YARN.error("AMRMClientAsync.onError() received {}", e, e); - queue(new ActionHalt(EXIT_EXCEPTION_THROWN, - SliderUtils.extractFirstLine(e.getLocalizedMessage()))); - } else { - // ignore and log - LOG_YARN.info("Ignoring AMRMClientAsync.onError() received {}", e); - } - } - -/* =================================================================== */ -/* RMOperationHandlerActions */ -/* =================================================================== */ - - - @Override - public void execute(List operations) { - rmOperationHandler.execute(operations); - } - - @Override - public void releaseAssignedContainer(ContainerId containerId) { - rmOperationHandler.releaseAssignedContainer(containerId); - } - - @Override - public void addContainerRequest(AMRMClient.ContainerRequest req) { - rmOperationHandler.addContainerRequest(req); - } - - @Override - public int cancelContainerRequests(Priority priority1, - Priority priority2, - int count) { - return rmOperationHandler.cancelContainerRequests(priority1, priority2, count); - } - - @Override - public void cancelSingleRequest(AMRMClient.ContainerRequest request) { - rmOperationHandler.cancelSingleRequest(request); - } - - @Override - public void updateBlacklist(List blacklistAdditions, - List blacklistRemovals) { - rmOperationHandler.updateBlacklist(blacklistAdditions, blacklistRemovals); - } - -/* =================================================================== */ -/* END */ -/* =================================================================== */ - - /** - * Launch the provider service - * @throws IOException - * @throws SliderException - */ - protected synchronized void launchProviderService() - throws IOException, SliderException { - // didn't start, so don't register - for (ProviderService providerService : providers) { -// providerService.start(); - } - // and send the started event ourselves - eventCallbackEvent(null); - } - - /* =================================================================== */ - /* EventCallback from the child or ourselves directly */ - /* =================================================================== */ - - @Override // ProviderCompleted - public void eventCallbackEvent(Object parameter) { - // now ask for the cluster nodes - try { - flexCluster(null); - } catch (Exception e) { - // cluster flex failure: log - log.error("Failed to flex cluster nodes: {}", e, e); - // then what? exit - queue(new ActionStopSlider(e)); - } - } - - - /** - * Async start container request - * @param container container - * @param ctx context - * @param instance node details - */ - public void startContainer(Container container, - ContainerLaunchContext ctx, - RoleInstance instance) throws IOException { - appState.containerStartSubmitted(container, instance); - - nmClientAsync.startContainerAsync(container, ctx); - } - - @Override // NMClientAsync.CallbackHandler - public void onContainerStopped(ContainerId containerId) { - // do nothing but log: container events from the AM - // are the source of container halt details to react to - log.info("onContainerStopped {} ", containerId); - } - - @Override // NMClientAsync.CallbackHandler - public void onContainerStarted(ContainerId containerId, - Map allServiceResponse) { - LOG_YARN.info("Started Container {} ", containerId); - RoleInstance cinfo = appState.onNodeManagerContainerStarted(containerId); - if (cinfo != null) { - LOG_YARN.info("Deployed instance of role {} onto {}", - cinfo.role, containerId); - //trigger an async container status - nmClientAsync.getContainerStatusAsync(containerId, - cinfo.container.getNodeId()); - // push out a registration - queue(new RegisterComponentInstance(containerId, cinfo, - 0, TimeUnit.MILLISECONDS)); - - } else { - //this is a hypothetical path not seen. We react by warning - log.error("Notified of started container that isn't pending {} - releasing", - containerId); - //then release it - asyncRMClient.releaseAssignedContainer(containerId); - } - } - - @Override // NMClientAsync.CallbackHandler - public void onStartContainerError(ContainerId containerId, Throwable t) { - LOG_YARN.error("Failed to start Container {}", containerId, t); - appState.onNodeManagerContainerStartFailed(containerId, t); - } - - @Override // NMClientAsync.CallbackHandler - public void onContainerStatusReceived(ContainerId containerId, - ContainerStatus containerStatus) { - LOG_YARN.debug("Container Status: id={}, status={}", containerId, - containerStatus); - RoleInstance cinfo = appState.getOwnedContainer(containerId); - if (cinfo == null) { - LOG_YARN.error("Owned container not found for {}", containerId); - return; - } - ProviderService providerService = ProviderFactory - .getProviderService(cinfo.providerRole.component.getArtifact()); -// if (providerService.processContainerStatus(containerId, containerStatus)) { -// try { -// Thread.sleep(1000); -// } catch (InterruptedException e) { -// } -// LOG_YARN.info("Re-requesting status for role {}, {}", -// cinfo.role, containerId); -// //trigger another async container status -// nmClientAsync.getContainerStatusAsync(containerId, -// cinfo.container.getNodeId()); -// } else if (timelineServiceEnabled) { -// RoleInstance instance = appState.getOwnedContainer(containerId); -// if (instance != null) { -// org.apache.slider.api.resource.Container container = -// instance.providerRole.component -// .getContainer(containerId.toString()); -// if (container != null) { -// serviceTimelinePublisher.componentInstanceUpdated(container); -// } -// } -// } - } - - @Override // NMClientAsync.CallbackHandler - public void onGetContainerStatusError( - ContainerId containerId, Throwable t) { - LOG_YARN.error("Failed to query the status of Container {}", containerId); - } - - @Override // NMClientAsync.CallbackHandler - public void onStopContainerError(ContainerId containerId, Throwable t) { - LOG_YARN.warn("Failed to stop Container {}", containerId); - } - - /** - * Queue an action for immediate execution in the executor thread - * @param action action to execute - */ - public void queue(AsyncAction action) { - actionQueues.put(action); - } - - /** - * Schedule an action - * @param action for delayed execution - */ - public void schedule(AsyncAction action) { - actionQueues.schedule(action); - } - - - /** - * Handle any exception in a thread. If the exception provides an exit - * code, that is the one that will be used - * @param thread thread throwing the exception - * @param exception exception - */ - public void onExceptionInThread(Thread thread, Throwable exception) { - log.error("Exception in {}: {}", thread.getName(), exception, exception); - - // if there is a teardown in progress, ignore it - if (amCompletionFlag.get()) { - log.info("Ignoring exception: shutdown in progress"); - } else { - int exitCode = EXIT_EXCEPTION_THROWN; - if (exception instanceof ExitCodeProvider) { - exitCode = ((ExitCodeProvider) exception).getExitCode(); - } - signalAMComplete(new ActionStopSlider("stop", - exitCode, - FinalApplicationStatus.FAILED, - SliderUtils.extractFirstLine(exception.getLocalizedMessage()))); - } - } - - /** - * TODO Read chaos monkey params from AM configuration rather than app - * configuration - * @return true if it started - */ - private boolean maybeStartMonkey() { - org.apache.slider.api.resource.Configuration configuration = - application.getConfiguration(); - boolean enabled = configuration.getPropertyBool( - InternalKeys.CHAOS_MONKEY_ENABLED, - InternalKeys.DEFAULT_CHAOS_MONKEY_ENABLED); - if (!enabled) { - log.debug("Chaos monkey disabled"); - return false; - } - - long monkeyInterval = SliderUtils.getTimeRange(configuration, - InternalKeys.CHAOS_MONKEY_INTERVAL, - InternalKeys.DEFAULT_CHAOS_MONKEY_INTERVAL_DAYS, - InternalKeys.DEFAULT_CHAOS_MONKEY_INTERVAL_HOURS, - InternalKeys.DEFAULT_CHAOS_MONKEY_INTERVAL_MINUTES, - 0); - if (monkeyInterval == 0) { - log.debug( - "Chaos monkey not configured with a time interval...not enabling"); - return false; - } - - long monkeyDelay = SliderUtils.getTimeRange(configuration, - InternalKeys.CHAOS_MONKEY_DELAY, - 0, - 0, - 0, - (int)monkeyInterval); - - log.info("Adding Chaos Monkey scheduled every {} seconds ({} hours -delay {}", - monkeyInterval, monkeyInterval/(60*60), monkeyDelay); - monkey = new ChaosMonkeyService(metrics, actionQueues); - initAndAddService(monkey); - - // configure the targets - - // launch failure: special case with explicit failure triggered now - int amLaunchFailProbability = configuration.getPropertyInt( - InternalKeys.CHAOS_MONKEY_PROBABILITY_AM_LAUNCH_FAILURE, - 0); - if (amLaunchFailProbability > 0 && monkey.chaosCheck( - amLaunchFailProbability)) { - log.info("Chaos Monkey has triggered AM Launch failure"); - // trigger a failure - ActionStopSlider stop = new ActionStopSlider("stop", - 0, TimeUnit.SECONDS, - LauncherExitCodes.EXIT_FALSE, - FinalApplicationStatus.FAILED, - E_TRIGGERED_LAUNCH_FAILURE); - queue(stop); - } - - int amKillProbability = configuration.getPropertyInt( - InternalKeys.CHAOS_MONKEY_PROBABILITY_AM_FAILURE, - InternalKeys.DEFAULT_CHAOS_MONKEY_PROBABILITY_AM_FAILURE); - monkey.addTarget("AM killer", - new ChaosKillAM(actionQueues, -1), amKillProbability); - int containerKillProbability = configuration.getPropertyInt( - InternalKeys.CHAOS_MONKEY_PROBABILITY_CONTAINER_FAILURE, - InternalKeys.DEFAULT_CHAOS_MONKEY_PROBABILITY_CONTAINER_FAILURE); - monkey.addTarget("Container killer", - new ChaosKillContainer(appState, actionQueues, rmOperationHandler), - containerKillProbability); - - // and schedule it - if (monkey.schedule(monkeyDelay, monkeyInterval, TimeUnit.SECONDS)) { - log.info("Chaos Monkey is running"); - return true; - } else { - log.info("Chaos monkey not started"); - return false; - } - } - - /** - * This is the main entry point for the service launcher. - * @param args command line arguments. - */ - public static void main(String[] args) { - - //turn the args to a list - List argsList = Arrays.asList(args); - //create a new list, as the ArrayList type doesn't push() on an insert - List extendedArgs = new ArrayList(argsList); - //insert the service name - extendedArgs.add(0, SERVICE_CLASSNAME); - //now have the service launcher do its work - ServiceLauncher.serviceMain(extendedArgs); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionFlexCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionFlexCluster.java deleted file mode 100644 index a7b94ed..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionFlexCluster.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.slider.api.proto.Messages; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; - -import java.util.concurrent.TimeUnit; - -public class ActionFlexCluster extends AsyncAction { - - final Messages.FlexComponentsRequestProto requestProto; - public ActionFlexCluster(String name, long delay, TimeUnit timeUnit, - Messages.FlexComponentsRequestProto requestProto) { - super(name, delay, timeUnit, ATTR_CHANGES_APP_SIZE); - this.requestProto = requestProto; - } - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - appMaster.flexCluster(requestProto); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionHalt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionHalt.java deleted file mode 100644 index ee1bb72..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionHalt.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.hadoop.util.ExitUtil; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; - -import java.util.concurrent.TimeUnit; - -/** - * Exit an emergency JVM halt. - * @see ExitUtil#halt(int, String) - */ -public class ActionHalt extends AsyncAction { - - private final int status; - private final String text; - - public ActionHalt( - int status, - String text) { - this(status, text, 0, TimeUnit.MILLISECONDS); - } - - public ActionHalt( - int status, - String text, - long delay, TimeUnit timeUnit) { - - // do not declare that this action halts the cluster ... keep it a surprise - super("Halt", delay, timeUnit); - this.status = status; - this.text = text; - } - - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - ExitUtil.halt(status, text); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionKillContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionKillContainer.java deleted file mode 100644 index 7446e82..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionKillContainer.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.ContainerReleaseOperation; -import org.apache.slider.server.appmaster.operations.RMOperationHandler; -import org.apache.slider.server.appmaster.operations.RMOperationHandlerActions; -import org.apache.slider.server.appmaster.state.AppState; - -import java.util.LinkedList; -import java.util.List; -import java.util.concurrent.TimeUnit; - -/** - * Kill a specific container - */ -public class ActionKillContainer extends AsyncAction { - - /** - * container to kill - */ - private final ContainerId containerId; - - /** - * handler for the operation - */ - private final RMOperationHandlerActions operationHandler; - - /** - * Kill a container - * @param containerId container to kill - * @param delay - * @param timeUnit - * @param operationHandler - */ - public ActionKillContainer( - ContainerId containerId, - long delay, - TimeUnit timeUnit, - RMOperationHandlerActions operationHandler) { - super("kill container", delay, timeUnit, ATTR_CHANGES_APP_SIZE); - this.operationHandler = operationHandler; - Preconditions.checkArgument(containerId != null); - - this.containerId = containerId; - } - - /** - * Get the container ID to kill - * @return - */ - public ContainerId getContainerId() { - return containerId; - } - - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - List opsList = new LinkedList<>(); - ContainerReleaseOperation release = new ContainerReleaseOperation(containerId); - opsList.add(release); - //now apply the operations - operationHandler.execute(opsList); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionRegisterServiceInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionRegisterServiceInstance.java deleted file mode 100644 index 0d7f7d4..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionRegisterServiceInstance.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.slider.api.resource.Application; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; - -import java.util.concurrent.TimeUnit; - -/** - * Asynchronous registration operation - */ -public class ActionRegisterServiceInstance extends AsyncAction { - - private final String instanceName; - private final ApplicationId appId; - private final Application application; - public ActionRegisterServiceInstance(String instanceName, - ApplicationId appId, Application application) { - super("ActionRegisterServiceInstance"); - this.instanceName = instanceName; - this.appId = appId; - this.application = application; - } - - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - - // YARN Registry do the registration - appMaster.registerServiceInstance(instanceName, appId, application); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStartContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStartContainer.java deleted file mode 100644 index 358c844..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStartContainer.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.RoleInstance; - -import java.util.Locale; -import java.util.concurrent.TimeUnit; - -/** - * Start a container - * @see SliderAppMaster#startContainer(Container, ContainerLaunchContext, RoleInstance) - */ -public class ActionStartContainer extends AsyncAction { - - private final Container container; - private final ContainerLaunchContext ctx; - private final RoleInstance instance; - - public ActionStartContainer(String name, - Container container, - ContainerLaunchContext ctx, - RoleInstance instance, - long delay, TimeUnit timeUnit) { - super( - String.format(Locale.ENGLISH, - "%s %s: /", - name , container.getId().toString()), - delay, - timeUnit); - this.container = container; - this.ctx = ctx; - this.instance = instance; - } - - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - appMaster.startContainer(container, ctx, instance); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopQueue.java deleted file mode 100644 index 08e8086..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopQueue.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.concurrent.TimeUnit; - -/** - * Action to tell a queue executor to stop -after handing this on/executing it - */ -public class ActionStopQueue extends AsyncAction { - private static final Logger log = - LoggerFactory.getLogger(ActionStopQueue.class); - - public ActionStopQueue(long delay) { - super("stop queue", delay); - } - - public ActionStopQueue(long delay, - TimeUnit timeUnit) { - super("stop queue", delay, timeUnit); - } - - public ActionStopQueue(String name, - long delay, - TimeUnit timeUnit) { - super(name, delay, timeUnit); - } - - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - log.warn("STOP"); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopSlider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopSlider.java deleted file mode 100644 index 055cea5..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopSlider.java +++ /dev/null @@ -1,162 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.slider.core.exceptions.ExceptionConverter; -import org.apache.slider.core.exceptions.TriggerClusterTeardownException; -import org.apache.slider.core.main.ExitCodeProvider; -import org.apache.slider.core.main.LauncherExitCodes; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; - -import java.util.concurrent.TimeUnit; - -/** - * Trigger an AM exit. This is used to build the exit status message for YARN - */ -public class ActionStopSlider extends AsyncAction { - - private int exitCode; - private FinalApplicationStatus finalApplicationStatus; - private String message; - private final Exception ex; - - /** - * Simple constructor - * @param name action name - */ - public ActionStopSlider(String name) { - super(name); - this.ex = null; - } - - /** - * Stop slider - * @param name action name - * @param delay execution delay - * @param timeUnit delay time unit - * @param exitCode process exit code - * @param finalApplicationStatus yarn status - * @param message message for AM - */ - public ActionStopSlider(String name, - long delay, - TimeUnit timeUnit, - int exitCode, - FinalApplicationStatus finalApplicationStatus, - String message) { - super(name, delay, timeUnit, ATTR_HALTS_APP); - this.exitCode = exitCode; - this.finalApplicationStatus = finalApplicationStatus; - this.message = message; - this.ex = null; - } - - /** - * Stop slider - * @param name action name - * @param exitCode process exit code - * @param finalApplicationStatus yarn status - * @param message message for AM - */ - public ActionStopSlider(String name, - int exitCode, - FinalApplicationStatus finalApplicationStatus, - String message) { - super(name); - this.exitCode = exitCode; - this.finalApplicationStatus = finalApplicationStatus; - this.message = message; - this.ex = null; - } - - /** - * Simple constructor - * @param ex teardown exception - */ - public ActionStopSlider(TriggerClusterTeardownException ex) { - this("stop", - ex.getExitCode(), - ex.getFinalApplicationStatus(), - ex.getMessage()); - } - - /** - * Build from an exception. - *

- * If the exception implements - * {@link ExitCodeProvider} then the exit code is extracted from that - * @param ex exception. - */ - public ActionStopSlider(Exception ex) { - super("stop"); - if (ex instanceof ExitCodeProvider) { - setExitCode(((ExitCodeProvider)ex).getExitCode()); - } else { - setExitCode(LauncherExitCodes.EXIT_EXCEPTION_THROWN); - } - setFinalApplicationStatus(FinalApplicationStatus.FAILED); - setMessage(ex.getMessage()); - this.ex = ex; - } - - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - SliderAppMaster.getLog().info("SliderAppMasterApi.stopCluster: {}", - message); - appMaster.onAMStop(this); - } - - @Override - public String toString() { - return String.format("%s: exit code = %d, %s: %s;", - name, exitCode, finalApplicationStatus, message) ; - } - - public int getExitCode() { - return exitCode; - } - - public void setExitCode(int exitCode) { - this.exitCode = exitCode; - } - - public FinalApplicationStatus getFinalApplicationStatus() { - return finalApplicationStatus; - } - - public void setFinalApplicationStatus(FinalApplicationStatus finalApplicationStatus) { - this.finalApplicationStatus = finalApplicationStatus; - } - - public String getMessage() { - return message; - } - - public void setMessage(String message) { - this.message = message; - } - - public Exception getEx() { - return ex; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionUpgradeContainers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionUpgradeContainers.java deleted file mode 100644 index 05fcbcc..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionUpgradeContainers.java +++ /dev/null @@ -1,106 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.TimeUnit; - -import org.apache.commons.collections.CollectionUtils; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; - -public class ActionUpgradeContainers extends AsyncAction { - private int exitCode; - private FinalApplicationStatus finalApplicationStatus; - private String message; - private Set containers = new HashSet<>(); - private Set components = new HashSet<>(); - - public ActionUpgradeContainers(String name, - long delay, - TimeUnit timeUnit, - int exitCode, - FinalApplicationStatus finalApplicationStatus, - List containers, - List components, - String message) { - super(name, delay, timeUnit); - this.exitCode = exitCode; - this.finalApplicationStatus = finalApplicationStatus; - this.containers.addAll(containers); - this.components.addAll(components); - this.message = message; - } - - @Override - public void execute(SliderAppMaster appMaster, QueueAccess queueService, - AppState appState) throws Exception { - if (CollectionUtils.isNotEmpty(this.containers) - || CollectionUtils.isNotEmpty(this.components)) { - SliderAppMaster.getLog().info("SliderAppMaster.upgradeContainers: {}", - message); - appMaster.onUpgradeContainers(this); - } - } - - public int getExitCode() { - return exitCode; - } - - public void setExitCode(int exitCode) { - this.exitCode = exitCode; - } - - public FinalApplicationStatus getFinalApplicationStatus() { - return finalApplicationStatus; - } - - public void setFinalApplicationStatus( - FinalApplicationStatus finalApplicationStatus) { - this.finalApplicationStatus = finalApplicationStatus; - } - - public String getMessage() { - return message; - } - - public void setMessage(String message) { - this.message = message; - } - - public Set getContainers() { - return containers; - } - - public void setContainers(Set containers) { - this.containers = containers; - } - - public Set getComponents() { - return components; - } - - public void setComponents(Set components) { - this.components = components; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/AsyncAction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/AsyncAction.java deleted file mode 100644 index f9a1fd5..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/AsyncAction.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; - -import java.io.IOException; -import java.util.concurrent.Delayed; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; - -public abstract class AsyncAction implements Delayed { - - private static final AtomicLong sequencer = new AtomicLong(0); - - public final String name; - private long nanos; - public final int attrs; - private final long sequenceNumber = sequencer.incrementAndGet(); - - - protected AsyncAction(String name) { - this(name, 0); - } - - protected AsyncAction(String name, - long delayMillis) { - this(name, delayMillis, TimeUnit.MILLISECONDS); - } - - protected AsyncAction(String name, - long delay, - TimeUnit timeUnit) { - this(name, delay, timeUnit, 0); - } - - protected AsyncAction(String name, - long delay, - TimeUnit timeUnit, - int attrs) { - this.name = name; - this.setNanos(convertAndOffset(delay, timeUnit)); - this.attrs = attrs; - } - - protected long convertAndOffset(long delay, TimeUnit timeUnit) { - return now() + TimeUnit.NANOSECONDS.convert(delay, timeUnit); - } - - /** - * The current time in nanos - * @return now - */ - protected long now() { - return System.nanoTime(); - } - - @Override - public long getDelay(TimeUnit unit) { - return unit.convert(getNanos() - now(), TimeUnit.NANOSECONDS); - } - - @Override - public int compareTo(Delayed that) { - if (this == that) { - return 0; - } - return SliderUtils.compareTo( - getDelay(TimeUnit.NANOSECONDS), - that.getDelay(TimeUnit.NANOSECONDS)); - } - - @Override - public String toString() { - final StringBuilder sb = - new StringBuilder(super.toString()); - sb.append(" name='").append(name).append('\''); - sb.append(", delay=").append(getDelay(TimeUnit.SECONDS)); - sb.append(", attrs=").append(attrs); - sb.append(", sequenceNumber=").append(sequenceNumber); - sb.append('}'); - return sb.toString(); - } - - protected int getAttrs() { - return attrs; - } - - /** - * Ask if an action has an of the specified bits set. - * This is not an equality test. - * @param attr attribute - * @return true iff the action has any of the bits in the attr arg set - */ - public boolean hasAttr(int attr) { - return (attrs & attr) != 0; - } - - /** - * Actual application - * @param appMaster - * @param queueService - * @param appState - * @throws IOException - */ - public abstract void execute(SliderAppMaster appMaster, - QueueAccess queueService, AppState appState) throws Exception; - - public long getNanos() { - return nanos; - } - - public void setNanos(long nanos) { - this.nanos = nanos; - } - - public static final int ATTR_CHANGES_APP_SIZE = 1; - public static final int ATTR_HALTS_APP = 2; - public static final int ATTR_REVIEWS_APP_SIZE = 4; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/EscalateOutstandingRequests.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/EscalateOutstandingRequests.java deleted file mode 100644 index 2c545ea..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/EscalateOutstandingRequests.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; - -import java.util.concurrent.TimeUnit; - -/** - * Escalate outstanding requests by asking AM - */ -public class EscalateOutstandingRequests extends AsyncAction { - - public EscalateOutstandingRequests() { - super("EscalateOutstandingRequests"); - } - - public EscalateOutstandingRequests(long delay, - TimeUnit timeUnit) { - super("EscalateOutstandingRequests", delay, timeUnit, ATTR_REVIEWS_APP_SIZE); - } - - @Override - public void execute(SliderAppMaster appMaster, QueueAccess queueService, AppState appState) throws - Exception { - appMaster.escalateOutstandingRequests(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/MonitorComponentInstances.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/MonitorComponentInstances.java deleted file mode 100644 index f7aa871..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/MonitorComponentInstances.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; - -/** - * Execute readiness checks on component instances. - */ -public class MonitorComponentInstances extends AsyncAction { - - public MonitorComponentInstances() { - super("MonitorComponentInstance"); - } - - @Override - public void execute(SliderAppMaster appMaster, QueueAccess queueService, - AppState appState) throws Exception { - appMaster.monitorComponentInstances(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ProviderStartupCompleted.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ProviderStartupCompleted.java deleted file mode 100644 index 957a35f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ProviderStartupCompleted.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; - -public class ProviderStartupCompleted extends AsyncAction { - - public ProviderStartupCompleted() { - super("ProviderStartupCompleted"); - } - - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - appMaster.eventCallbackEvent(null); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueAccess.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueAccess.java deleted file mode 100644 index 0396891..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueAccess.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -/** - * Access for queue operations - */ -public interface QueueAccess { - /** - * Put an action on the immediate queue -to be executed when the queue - * reaches it. - * @param action action to queue - */ - void put(AsyncAction action); - - /** - * Put a delayed action: this will only be added to the main queue - * after its action time has been reached - * @param action action to queue - */ - void schedule(AsyncAction action); - - /** - * Remove an action from the queues. - * @param action action to remove - * @return true if the action was removed - */ - boolean remove(AsyncAction action); - - /** - * Add a named renewing action - * @param name name - * @param renewingAction wrapped action - */ - void renewing(String name, - RenewingAction renewingAction); - - /** - * Look up a renewing action - * @param name name of the action - * @return the action or null if none was found - */ - RenewingAction lookupRenewingAction(String name); - - /** - * Remove a renewing action - * @param name action name name of the action - * @return true if the action was found and removed. - */ - boolean removeRenewingAction(String name); - - /** - * Look in the immediate queue for any actions of a specific attribute - */ - boolean hasQueuedActionWithAttribute(int attr); -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueExecutor.java deleted file mode 100644 index d0fc2cf..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueExecutor.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * Executor for async actions - hands them off to the AM as - * appropriate - */ -public class QueueExecutor implements Runnable { - private static final Logger log = - LoggerFactory.getLogger(QueueExecutor.class); - - private final SliderAppMaster appMaster; - private final QueueService actionQueues; - private final AppState appState; - - - public QueueExecutor(SliderAppMaster appMaster, - QueueService actionQueues) { - Preconditions.checkNotNull(appMaster); - Preconditions.checkNotNull(actionQueues); - - this.appMaster = appMaster; - this.actionQueues = actionQueues; - this.appState = appMaster.getAppState(); - } - - @VisibleForTesting - public QueueExecutor(QueueService actionQueues) { - Preconditions.checkNotNull(actionQueues); - this.appMaster = null; - this.appState = null; - this.actionQueues = actionQueues; - } - - /** - * Run until the queue has been told to stop - */ - @Override - public void run() { - AsyncAction take = null; - try { - log.info("Queue Executor run() started"); - do { - take = actionQueues.actionQueue.take(); - log.debug("Executing {}", take); - - take.execute(appMaster, actionQueues, appState); - log.debug("Completed {}", take); - - } while (!(take instanceof ActionStopQueue)); - log.info("Queue Executor run() stopped"); - } catch (InterruptedException e) { - // interrupted: exit - } catch (Throwable e) { - log.error("Exception processing {}: {}", take, e, e); - if (appMaster != null) { - appMaster.onExceptionInThread(Thread.currentThread(), e); - } - } - // tag completed - actionQueues.complete(); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueService.java deleted file mode 100644 index 34acade..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueService.java +++ /dev/null @@ -1,202 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - - -import org.apache.slider.server.services.workflow.ServiceThreadFactory; -import org.apache.slider.server.services.workflow.WorkflowExecutorService; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Iterator; -import java.util.Map; -import java.util.concurrent.BlockingDeque; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.DelayQueue; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingDeque; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * The Queue service provides immediate and scheduled queues, as well - * as an executor thread that moves queued actions from the scheduled - * queue to the immediate one. - * - *

- * This code to be revisited to see if all that was needed is the single scheduled - * queue, implicitly making actions immediate by giving them an execution - * time of "now". It would force having a sequence number to all actions, one - * which the queue would have to set from its (monotonic, thread-safe) counter - * on every submission, with a modified comparison operator. This would guarantee - * that earlier submissions were picked before later ones. - */ -public class QueueService extends WorkflowExecutorService -implements Runnable, QueueAccess { - private static final Logger log = - LoggerFactory.getLogger(QueueService.class); - public static final String NAME = "Action Queue"; - private final AtomicBoolean completed = new AtomicBoolean(false); - - /** - * Immediate actions. - */ - public final BlockingDeque actionQueue = - new LinkedBlockingDeque<>(); - - /** - * Actions to be scheduled in the future - */ - public final DelayQueue scheduledActions = new DelayQueue<>(); - - /** - * Map of renewing actions by name ... this is to allow them to - * be cancelled by name - */ - private final Map> renewingActions - = new ConcurrentHashMap<>(); - - /** - * Create a queue instance with a single thread executor - */ - public QueueService() { - super(NAME, - ServiceThreadFactory.singleThreadExecutor(NAME, true)); - } - - @Override - public void put(AsyncAction action) { - log.debug("Queueing {}", action); - actionQueue.add(action); - } - - @Override - public void schedule(AsyncAction action) { - log.debug("Scheduling {}", action); - scheduledActions.add(action); - } - - @Override - public boolean remove(AsyncAction action) { - boolean removedFromDelayQueue = scheduledActions.remove(action); - boolean removedFromActions = actionQueue.remove(action); - return removedFromActions || removedFromDelayQueue; - } - - @Override - public void renewing(String name, - RenewingAction renewingAction) { - log.debug("Adding renewing Action \"{}\": {}", name, - renewingAction.getAction()); - if (removeRenewingAction(name)) { - log.debug("Removed predecessor action"); - } - renewingActions.put(name, renewingAction); - schedule(renewingAction); - } - - @Override - public RenewingAction lookupRenewingAction(String name) { - return renewingActions.get(name); - } - - @Override - public boolean removeRenewingAction(String name) { - RenewingAction action = renewingActions.remove(name); - return action != null && remove(action); - } - - /** - * Stop the service by scheduling an {@link ActionStopQueue} action - * ..if the processor thread is working this will propagate through - * and stop the queue handling after all other actions complete. - * @throws Exception - */ - @Override - protected void serviceStop() throws Exception { - ActionStopQueue stopQueue = new ActionStopQueue("serviceStop: "+ this, - 0, TimeUnit.MILLISECONDS); - schedule(stopQueue); - super.serviceStop(); - } - - /** - * Flush an action queue of all types of a specific action - * @param clazz - */ - protected void flushActionQueue(Class clazz) { - Iterator iterator = - actionQueue.descendingIterator(); - while (iterator.hasNext()) { - AsyncAction next = iterator.next(); - if (next.getClass().equals(clazz)) { - iterator.remove(); - } - } - } - - @Override - public boolean hasQueuedActionWithAttribute(int attr) { - for (AsyncAction action : actionQueue) { - if (action.hasAttr(attr)) { - return true; - } - } - return false; - } - - /** - * Run until the queue has been told to stop - */ - @Override - public void run() { - try { - - log.info("QueueService processor started"); - - AsyncAction take; - do { - take = scheduledActions.take(); - log.debug("Propagating {}", take); - actionQueue.put(take); - } while (!(take instanceof ActionStopQueue)); - log.info("QueueService processor terminated"); - } catch (InterruptedException e) { - // interrupted during actions - } - // the thread exits, but does not tag the service as complete. That's expected - // to be done by the stop queue - } - - - /** - * Check to see if the queue executor has completed - * @return the status - */ - public boolean isCompleted() { - return completed.get(); - } - - /** - * Package scoped method to mark the queue service as finished - */ - void complete() { - completed.set(true); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RegisterComponentInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RegisterComponentInstance.java deleted file mode 100644 index 3c1bed8..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RegisterComponentInstance.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.RoleInstance; - -import java.util.concurrent.TimeUnit; - -/** - * Notify the app master that it should register a component instance - * in the registry - * {@link SliderAppMaster#registerComponent(ContainerId)} - */ -public class RegisterComponentInstance extends AsyncAction { - - public final ContainerId containerId; - public final RoleInstance roleInstance; - - public RegisterComponentInstance(ContainerId containerId, - RoleInstance roleInstance, - long delay, - TimeUnit timeUnit) { - super("RegisterComponentInstance :" + containerId, - delay, timeUnit); - this.roleInstance = roleInstance; - Preconditions.checkArgument(containerId != null); - this.containerId = containerId; - } - - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - - appMaster.registerComponent(containerId, roleInstance); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RenewingAction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RenewingAction.java deleted file mode 100644 index f3143ea..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RenewingAction.java +++ /dev/null @@ -1,141 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import com.google.common.base.Preconditions; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantReadWriteLock; - -/** - * This action executes then reschedules an inner action; a limit - * can specify the number of times to run - */ - -public class RenewingAction extends AsyncAction { - private static final Logger log = - LoggerFactory.getLogger(RenewingAction.class); - private final A action; - private long interval; - private TimeUnit timeUnit; - public final AtomicInteger executionCount = new AtomicInteger(); - private final ReentrantReadWriteLock intervalLock = new ReentrantReadWriteLock(); - private final Lock intervalReadLock = intervalLock.readLock(); - private final Lock intervalWriteLock = intervalLock.writeLock(); - public final int limit; - - - /** - * Rescheduling action - * @param action action to execute - * @param initialDelay initial delay - * @param interval interval for later delays - * @param timeUnit time unit for all times - * @param limit limit on the no. of executions. If 0 or less: no limit - */ - public RenewingAction(A action, - long initialDelay, - long interval, - TimeUnit timeUnit, - int limit) { - super("renewing " + action.name, initialDelay, timeUnit, action.getAttrs()); - Preconditions.checkArgument(interval > 0, "invalid interval: " + interval); - this.action = action; - this.interval = interval; - this.timeUnit = timeUnit; - this.limit = limit; - } - - /** - * Execute the inner action then reschedule ourselves - * @param appMaster - * @param queueService - * @param appState - * @throws Exception - */ - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) - throws Exception { - long exCount = executionCount.incrementAndGet(); - log.debug("{}: Executing inner action count # {}", this, exCount); - action.execute(appMaster, queueService, appState); - boolean reschedule = true; - if (limit > 0) { - reschedule = limit > exCount; - } - if (reschedule) { - this.setNanos(convertAndOffset(getInterval(), getTimeUnit())); - log.debug("{}: rescheduling, new offset {} mS ", this, - getDelay(TimeUnit.MILLISECONDS)); - queueService.schedule(this); - } - } - - /** - * Get the action - * @return - */ - public A getAction() { - return action; - } - - public long getInterval() { - intervalReadLock.lock(); - try { - return interval; - } finally { - intervalReadLock.unlock(); - } - } - - public void updateInterval(long delay, TimeUnit timeUnit) { - intervalWriteLock.lock(); - try { - interval = delay; - this.timeUnit = timeUnit; - } finally { - intervalWriteLock.unlock(); - } - } - - public TimeUnit getTimeUnit() { - intervalReadLock.lock(); - try { - return timeUnit; - } finally { - intervalReadLock.unlock(); - } - } - - public int getExecutionCount() { - return executionCount.get(); - } - - public int getLimit() { - return limit; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ResetFailureWindow.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ResetFailureWindow.java deleted file mode 100644 index 36f58dd..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ResetFailureWindow.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.RMOperationHandlerActions; -import org.apache.slider.server.appmaster.state.AppState; - -/** - * Requests the AM to reset the failure window - */ -public class ResetFailureWindow extends AsyncAction { - private final RMOperationHandlerActions operationHandler; - - public ResetFailureWindow(RMOperationHandlerActions operationHandler) { - super("ResetFailureWindow"); - this.operationHandler = operationHandler; - } - - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - synchronized (appMaster) { - appState.resetFailureCounts(); - AbstractRMOperation blacklistOperation = appState.updateBlacklist(); - if (blacklistOperation != null) { - blacklistOperation.execute(operationHandler); - } - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ReviewAndFlexApplicationSize.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ReviewAndFlexApplicationSize.java deleted file mode 100644 index bf7edf9..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ReviewAndFlexApplicationSize.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; - -import java.util.concurrent.TimeUnit; - -/** - * Tell the AM to execute the full flex review operation - */ -public class ReviewAndFlexApplicationSize extends AsyncAction { - - public ReviewAndFlexApplicationSize(String name, - long delay, - TimeUnit timeUnit) { - super(name, delay, timeUnit, ATTR_REVIEWS_APP_SIZE); - } - - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - appMaster.handleReviewAndFlexApplicationSize(this); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/UnregisterComponentInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/UnregisterComponentInstance.java deleted file mode 100644 index ac86333..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/UnregisterComponentInstance.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.RoleInstance; - -import java.util.concurrent.TimeUnit; - -/** - * Tell AM to unregister this component instance - */ -public class UnregisterComponentInstance extends AsyncAction { - - - public final RoleInstance roleInstance; - - public UnregisterComponentInstance(long delay, TimeUnit timeUnit, - RoleInstance roleInstance) { - super("UnregisterComponentInstance :" + roleInstance.getCompInstanceName() - + ", ContainerId = " + roleInstance.getContainerId(), - delay, timeUnit); - this.roleInstance = roleInstance; - } - - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - appMaster.unregisterComponent(roleInstance); - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetric.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetric.java deleted file mode 100644 index 89dfbfd..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetric.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.slider.server.appmaster.management; - -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Metric; - -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * A bool metric, mapped to an integer. true maps to 1, false to zero, - */ -public class BoolMetric implements Metric, Gauge { - - private final AtomicBoolean value; - - public BoolMetric(boolean b) { - value = new AtomicBoolean(b); - } - - public void set(boolean b) { - value.set(b); - } - - public boolean get() { - return value.get(); - } - - @Override - public Integer getValue() { - return value.get() ? 1 : 0; - } - - @Override - public String toString() { - return value.toString(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - BoolMetric that = (BoolMetric) o; - return get() == that.get(); - } - - @Override - public int hashCode() { - return value.hashCode(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetricPredicate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetricPredicate.java deleted file mode 100644 index 82bcd3a..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetricPredicate.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.slider.server.appmaster.management; - -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Metric; - -/** - * A metric which takes a predicate and returns 1 if the predicate evaluates - * to true. The predicate is evaluated whenever the metric is read. - */ -public class BoolMetricPredicate implements Metric, Gauge { - - private final Eval predicate; - - public BoolMetricPredicate(Eval predicate) { - this.predicate = predicate; - } - - @Override - public Integer getValue() { - return predicate.eval() ? 1: 0; - } - - public interface Eval { - boolean eval(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongGauge.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongGauge.java deleted file mode 100644 index c93467b..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongGauge.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * 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.slider.server.appmaster.management; - -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Metric; - -import java.util.concurrent.atomic.AtomicLong; - -/** - * This is a {@link AtomicLong} which acts as a metrics gauge: its state can be exposed as - * a metrics. - * It also exposes some of the same method names as the Codahale Counter class, so that - * it's easy to swap in. - * - */ -public class LongGauge extends AtomicLong implements Metric, Gauge { - - /** - * Instantiate - * @param val current value - */ - public LongGauge(long val) { - super(val); - } - - /** - * Instantiate with value 0 - */ - public LongGauge() { - this(0); - } - - /** - * Get the value as a metric - * @return current value - */ - @Override - public Long getValue() { - return get(); - } - - /** - * Method from {@Code counter}; used here for drop-in replacement - * without any recompile - * @return current value - */ - public Long getCount() { - return get(); - } - - /** - * {@code ++} - */ - public void inc() { - incrementAndGet(); - } - - /** - * {@code --} - */ - public void dec() { - decrementAndGet(); - } - - /** - * Decrement to the floor of 0. Operations in parallel may cause confusion here, - * but it will still never go below zero - * @param delta delta - * @return the current value - */ - public long decToFloor(long delta) { - long l = get(); - long r = l - delta; - if (r < 0) { - r = 0; - } - // if this fails, the decrement has been lost - compareAndSet(l, r); - return get(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongMetricFunction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongMetricFunction.java deleted file mode 100644 index 1de7345..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongMetricFunction.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.slider.server.appmaster.management; - -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Metric; - -/** - * A metric which takes a function to generate a long value. - * The function is evaluated whenever the metric is read. - */ -public class LongMetricFunction implements Metric, Gauge { - - private final Eval function; - - public LongMetricFunction(Eval function) { - this.function = function; - } - - @Override - public Long getValue() { - return function.eval(); - } - - public interface Eval { - long eval(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MeterAndCounter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MeterAndCounter.java deleted file mode 100644 index 02ab7bc..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MeterAndCounter.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * 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.slider.server.appmaster.management; - -import com.codahale.metrics.Counter; -import com.codahale.metrics.Meter; -import com.codahale.metrics.MetricRegistry; - -/** - * A combined meter and counter that can be used to measure load. - * Hash and equality are derived from the name - */ -public class MeterAndCounter { - - /** - * suffix for counters: {@value} - */ - public static final String COUNTER = ".counter"; - - /** - * suffix for meters: {@value} - */ - public static final String METER = ".meter"; - - final Meter meter; - final Counter counter; - final String name; - - /** - * Construct an instance - * @param metrics metrics to bond to - * @param name name before suffixes are appended - */ - public MeterAndCounter(MetricRegistry metrics, String name) { - this.name = name; - counter = metrics.counter(name + COUNTER); - meter = metrics.meter(name + METER); - } - - /** - * Construct an instance - * @param metrics metrics to bond to - * @param clazz class to use to derive name - * @param name name before suffixes are appended - */ - - public MeterAndCounter(MetricRegistry metrics, Class clazz, String name) { - this.name = name; - counter = metrics.counter(MetricRegistry.name(clazz, name + COUNTER)); - meter = metrics.meter(MetricRegistry.name(clazz, name + METER)); - } - - /** - * Increment the counter, mark the meter - */ - public void mark() { - counter.inc(); - meter.mark(); - } - - public void inc() { - mark(); - } - - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - MeterAndCounter that = (MeterAndCounter) o; - - return name.equals(that.name); - - } - - @Override - public int hashCode() { - return name.hashCode(); - } - - /** - * Get the count. - * @return the current count - */ - public long getCount() { - return counter.getCount(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java deleted file mode 100644 index 1fe8ea6..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java +++ /dev/null @@ -1,170 +0,0 @@ -/* - * 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.slider.server.appmaster.management; - -import com.codahale.metrics.Metric; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.MetricSet; -import com.codahale.metrics.health.HealthCheckRegistry; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.CompositeService; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -/** - * Class for all metrics and monitoring - */ -public class MetricsAndMonitoring extends CompositeService { - protected static final Logger log = - LoggerFactory.getLogger(MetricsAndMonitoring.class); - public MetricsAndMonitoring(String name) { - super(name); - } - - public MetricsAndMonitoring() { - super("MetricsAndMonitoring"); - } - - /** - * Singleton of metrics registry - */ - final MetricRegistry metrics = new MetricRegistry(); - - final HealthCheckRegistry health = new HealthCheckRegistry(); - - private final Map meterAndCounterMap - = new ConcurrentHashMap<>(); - - private final List metricSets = new ArrayList<>(); - - public static final int EVENT_LIMIT = 1000; - - public MetricRegistry getMetrics() { - return metrics; - } - - public HealthCheckRegistry getHealth() { - return health; - } - - @Override - protected void serviceInit(Configuration conf) throws Exception { - addService(new MetricsBindingService("MetricsBindingService", - metrics)); - super.serviceInit(conf); - } - - @Override - protected void serviceStop() throws Exception { - super.serviceStop(); - for (MetricSet set : metricSets) { - unregister(set); - } - } - - public MeterAndCounter getMeterAndCounter(String name) { - return meterAndCounterMap.get(name); - } - - /** - * Get or create the meter/counter pair - * @param name name of instance - * @return an instance - */ - public MeterAndCounter getOrCreateMeterAndCounter(String name) { - MeterAndCounter instance = meterAndCounterMap.get(name); - if (instance == null) { - synchronized (this) { - // check in a sync block - instance = meterAndCounterMap.get(name); - if (instance == null) { - instance = new MeterAndCounter(metrics, name); - meterAndCounterMap.put(name, instance); - } - } - } - return instance; - } - - /** - * Get a specific meter and mark it. This will create and register it on demand. - * @param name name of meter/counter - */ - public void markMeterAndCounter(String name) { - MeterAndCounter meter = getOrCreateMeterAndCounter(name); - meter.mark(); - } - - /** - * Given a {@link Metric}, registers it under the given name. - * - * @param name the name of the metric - * @param metric the metric - * @param the type of the metric - * @return {@code metric} - * @throws IllegalArgumentException if the name is already registered - */ - public T register(String name, T metric) throws IllegalArgumentException { - return metrics.register(name, metric); - } - - public T register(Class klass, T metric, String... names) - throws IllegalArgumentException { - return register(MetricRegistry.name(klass, names), metric); - } - - /** - * Add a metric set for registering and deregistration on service stop - * @param metricSet metric set - */ - public void addMetricSet(MetricSet metricSet) { - metricSets.add(metricSet); - metrics.registerAll(metricSet); - } - - /** - * add a metric set, giving each entry a prefix - * @param prefix prefix (a trailing "." is automatically added) - * @param metricSet the metric set to register - */ - public void addMetricSet(String prefix, MetricSet metricSet) { - addMetricSet(new PrefixedMetricsSet(prefix, metricSet)); - } - - /** - * Unregister a metric set; robust - * @param metricSet metric set to unregister - */ - public void unregister(MetricSet metricSet) { - for (String s : metricSet.getMetrics().keySet()) { - try { - metrics.remove(s); - } catch (IllegalArgumentException e) { - // log but continue - log.info("Exception when trying to unregister {}", s, e); - } - } - } -} - diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsBindingService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsBindingService.java deleted file mode 100644 index 864a1cf..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsBindingService.java +++ /dev/null @@ -1,151 +0,0 @@ -/* - * 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.slider.server.appmaster.management; - -import com.codahale.metrics.JmxReporter; -import com.codahale.metrics.Metric; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.MetricSet; -import com.codahale.metrics.ScheduledReporter; -import com.codahale.metrics.Slf4jReporter; -import com.google.common.base.Preconditions; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.CompositeService; -import org.apache.slider.server.services.workflow.ClosingService; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -/** - * YARN service which hooks up Codahale metrics to - * JMX, and, if enabled Ganglia and/or an SLF4J log. - */ -public class MetricsBindingService extends CompositeService - implements MetricsKeys { - protected static final Logger log = - LoggerFactory.getLogger(MetricsBindingService.class); - private final MetricRegistry metrics; - - private String reportingDetails = "not started"; - - - public MetricsBindingService(String name, - MetricRegistry metrics) { - super(name); - Preconditions.checkArgument(metrics != null, "Null metrics"); - this.metrics = metrics; - } - - /** - * Instantiate...create a metric registry in the process - * @param name service name - */ - public MetricsBindingService(String name) { - this(name, new MetricRegistry()); - } - - /** - * Accessor for the metrics instance - * @return the metrics - */ - public MetricRegistry getMetrics() { - return metrics; - } - - @Override - protected void serviceStart() throws Exception { - super.serviceStart(); - - StringBuilder summary = new StringBuilder(); - Configuration conf = getConfig(); - - summary.append("Reporting to JMX"); - // always start the JMX binding - JmxReporter jmxReporter; - jmxReporter = JmxReporter.forRegistry(metrics).build(); - jmxReporter.start(); - addService(new ClosingService<>(jmxReporter)); - - - // Ganglia - if (conf.getBoolean(METRICS_GANGLIA_ENABLED, false)) { - log.warn("Ganglia integration is not implemented"); -/* - // This is all disabled due to transitive dependencies on an LGPL library - com.codahale.metrics.ganglia.GangliaReporter gangliaReporter; - String host = conf.getTrimmed(METRICS_GANGLIA_HOST, ""); - int port = conf.getInt(METRICS_GANGLIA_PORT, DEFAULT_GANGLIA_PORT); - int interval = conf.getInt(METRICS_GANGLIA_REPORT_INTERVAL, 60); - int ttl = 1; - info.ganglia.gmetric4j.gmetric.GMetric.UDPAddressingMode - mcast = info.ganglia.gmetric4j.gmetric.GMetric.UDPAddressingMode.getModeForAddress(host); - boolean ganglia31 = conf.getBoolean(METRICS_GANGLIA_VERSION_31, true); - - final info.ganglia.gmetric4j.gmetric.GMetric ganglia = - new info.ganglia.gmetric4j.gmetric.GMetric( - host, - port, - mcast, - ttl, - ganglia31); - gangliaReporter = com.codahale.metrics.ganglia.GangliaReporter.forRegistry(metrics) - .convertRatesTo(TimeUnit.SECONDS) - .convertDurationsTo( - TimeUnit.MILLISECONDS) - .build(ganglia); - gangliaReporter.start(interval, TimeUnit.SECONDS); - addService(new ClosingService(gangliaReporter)); - summary.append(String.format(", Ganglia at %s:%d interval=%d", - host, port, interval)); - */ - } - - // Logging - if (conf.getBoolean(METRICS_LOGGING_ENABLED, false)) { - ScheduledReporter reporter; - String logName = - conf.getTrimmed(METRICS_LOGGING_LOG, METRICS_DEFAULT_LOG); - int interval = conf.getInt(METRICS_LOGGING_LOG_INTERVAL, - METRICS_DEFAULT_LOG_INTERVAL); - reporter = Slf4jReporter.forRegistry(metrics) - .convertRatesTo(TimeUnit.SECONDS) - .outputTo(LoggerFactory.getLogger(logName)) - .convertDurationsTo(TimeUnit.MILLISECONDS) - .build(); - reporter.start(interval, TimeUnit.MINUTES); - addService(new ClosingService<>(reporter)); - summary.append(String.format(", SLF4J to log %s interval=%d", - logName, interval)); - } - reportingDetails = summary.toString(); - log.info(reportingDetails); - } - - - @Override - public String toString() { - return super.toString() + " " + reportingDetails; - } - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsConstants.java deleted file mode 100644 index fa6bfc0..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsConstants.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.slider.server.appmaster.management; - -/** - * Constants used in slider for metrics registration and lookup - */ -public class MetricsConstants { - - /** - * {@value} - */ - public static final String CONTAINERS_OUTSTANDING_REQUESTS = "containers.outstanding-requests"; - - /** - * {@value} - */ - public static final String CONTAINERS_STARTED = "containers.started"; - - /** - * {@value} - */ - public static final String CONTAINERS_SURPLUS = "containers.surplus"; - - /** - * {@value} - */ - public static final String CONTAINERS_COMPLETED = "containers.completed"; - - /** - * {@value} - */ - public static final String CONTAINERS_FAILED = "containers.failed"; - - /** - * {@value} - */ - public static final String CONTAINERS_START_FAILED = "containers.start-failed"; - - public static final String PREFIX_SLIDER_ROLES = "slider.roles."; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsKeys.java deleted file mode 100644 index 13b3b6b..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsKeys.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * 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.slider.server.appmaster.management; - -public interface MetricsKeys { - - /** - * Prefix for metrics configuration options: {@value} - */ - String METRICS_PREFIX = "slider.metrics."; - - /** - * Boolean to enable Ganglia metrics reporting - * {@value} - */ - String METRICS_GANGLIA_ENABLED = - METRICS_PREFIX + "ganglia.enabled"; - /** - * {@value} - */ - String METRICS_GANGLIA_HOST = METRICS_PREFIX + "ganglia.host"; - /** - * {@value} - */ - String METRICS_GANGLIA_PORT = METRICS_PREFIX + "ganglia.port"; - /** - * {@value} - */ - String METRICS_GANGLIA_VERSION_31 = METRICS_PREFIX + "ganglia.version-31"; - /** - * {@value} - */ - String METRICS_GANGLIA_REPORT_INTERVAL = METRICS_PREFIX + "ganglia.report.interval"; - /** - * {@value} - */ - int DEFAULT_GANGLIA_PORT = 8649; - - - /** - * Boolean to enable Logging metrics reporting - * {@value} - */ - String METRICS_LOGGING_ENABLED = - METRICS_PREFIX + "logging.enabled"; - - /** - * String name of log to log to - * {@value} - */ - String METRICS_LOGGING_LOG = - METRICS_PREFIX + "logging.log.name"; - - /** - * Default log name: {@value} - */ - String METRICS_DEFAULT_LOG = - "org.apache.slider.metrics.log"; - - - /** - * Int log interval in seconds - * {@value} - */ - String METRICS_LOGGING_LOG_INTERVAL = - METRICS_PREFIX + "logging.interval.minutes"; - - - /** - * Default log interval: {@value}. - * This is a big interval as in a long lived service, log overflows are easy - * to create. - */ - int METRICS_DEFAULT_LOG_INTERVAL = 60; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/PrefixedMetricsSet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/PrefixedMetricsSet.java deleted file mode 100644 index e9ad46a..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/PrefixedMetricsSet.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.slider.server.appmaster.management; - -import com.codahale.metrics.Metric; -import com.codahale.metrics.MetricSet; - -import java.util.HashMap; -import java.util.Map; - -/** - * From an existing metrics set, generate a new metrics set with the - * prefix in front of every key. - * - * The prefix is added directly: if you want a '.' between prefix and metric - * keys, include it in the prefix. - */ -public class PrefixedMetricsSet implements MetricSet { - - private final String prefix; - private final MetricSet source; - - public PrefixedMetricsSet(String prefix, MetricSet source) { - this.prefix = prefix; - this.source = source; - } - - @Override - public Map getMetrics() { - Map sourceMetrics = source.getMetrics(); - Map metrics = new HashMap<>(sourceMetrics.size()); - for (Map.Entry entry : sourceMetrics.entrySet()) { - metrics.put(prefix + "." + entry.getKey(), entry.getValue()); - } - return metrics; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/Timestamp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/Timestamp.java deleted file mode 100644 index c30e749..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/Timestamp.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.slider.server.appmaster.management; - -/** - * A timestamp metric - */ -public class Timestamp extends LongGauge { - - public Timestamp(long val) { - super(val); - } - - public Timestamp() { - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/YarnServiceHealthCheck.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/YarnServiceHealthCheck.java deleted file mode 100644 index 936563c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/YarnServiceHealthCheck.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.slider.server.appmaster.management; - -import com.codahale.metrics.health.HealthCheck; -import org.apache.hadoop.service.Service; - -public class YarnServiceHealthCheck extends HealthCheck { - - private final Service service; - - public YarnServiceHealthCheck(Service service) { - this.service = service; - } - - @Override - protected Result check() throws Exception { - return service.isInState(Service.STATE.STARTED) - ? Result.healthy() - : Result.unhealthy("Service is not running: %s", service); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java deleted file mode 100644 index a397e19..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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.slider.server.appmaster.monkey; - -import com.codahale.metrics.Counter; -import com.codahale.metrics.MetricRegistry; -import com.google.common.base.Preconditions; -import org.apache.commons.lang.StringUtils; -import org.apache.slider.api.InternalKeys; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Entry in the chaos list - */ -public class ChaosEntry { - - protected static final Logger log = - LoggerFactory.getLogger(ChaosEntry.class); - public final String name; - public final ChaosTarget target; - public final long probability; - - private final Counter invocationCounter; - - - /** - * Constructor -includes validation of all arguments - * @param name entry name - * @param target target - * @param probability probability of occurring - */ - public ChaosEntry(String name, ChaosTarget target, long probability, - MetricRegistry metrics) { - Preconditions.checkArgument(!StringUtils.isEmpty(name), "missing name"); - Preconditions.checkArgument(target != null, "null target"); - Preconditions.checkArgument(probability > 0, "negative probability"); - Preconditions.checkArgument(probability <= InternalKeys.PROBABILITY_PERCENT_100, - "probability over 100%: "+ probability); - this.name = name; - this.target = target; - this.probability = probability; - invocationCounter = - metrics.counter(MetricRegistry.name(ChaosEntry.class, name)); - } - - /** - * Trigger the chaos action - */ - public void invokeChaos() { - log.info("Invoking {}", name); - invocationCounter.inc(); - target.chaosAction(); - } - - /** - * Invoke Chaos if the trigger value is in range of the probability - * @param value trigger value, 0-10K - * @return true if the chaos method was invoked - */ - public boolean maybeInvokeChaos(long value) { - log.debug("Probability {} trigger={}", probability, value); - if (value < probability) { - invokeChaos(); - return true; - } - return false; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillAM.java deleted file mode 100644 index 3c1a914..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillAM.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.slider.server.appmaster.monkey; - -import org.apache.slider.server.appmaster.actions.ActionHalt; -import org.apache.slider.server.appmaster.actions.QueueAccess; - -import java.util.concurrent.TimeUnit; - -/** - * Kill the AM - */ -public class ChaosKillAM implements ChaosTarget { - - public static final int DELAY = 1000; - private final QueueAccess queues; - private final int exitCode; - - public ChaosKillAM(QueueAccess queues, int exitCode) { - this.queues = queues; - this.exitCode = exitCode; - } - - /** - * Trigger a delayed halt - */ - @Override - public void chaosAction() { - queues.schedule(new ActionHalt(exitCode, "Chaos invoked halt", DELAY, - TimeUnit.MILLISECONDS)); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillContainer.java deleted file mode 100644 index 022312c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillContainer.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * 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.slider.server.appmaster.monkey; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.slider.server.appmaster.actions.ActionKillContainer; -import org.apache.slider.server.appmaster.actions.QueueAccess; -import org.apache.slider.server.appmaster.operations.RMOperationHandler; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; -import java.util.ListIterator; -import java.util.Random; -import java.util.concurrent.TimeUnit; - -/** - * Kill a container - */ -public class ChaosKillContainer implements ChaosTarget { - protected static final Logger log = - LoggerFactory.getLogger(ChaosKillContainer.class); - public static final int DELAY = 100; - private final AppState appState; - private final QueueAccess queues; - private final Random random = new Random(); - private final RMOperationHandler operationHandler; - - public ChaosKillContainer(AppState appState, - QueueAccess queues, - RMOperationHandler operationHandler) { - Preconditions.checkNotNull(appState); - Preconditions.checkNotNull(queues); - this.appState = appState; - this.queues = queues; - this.operationHandler = operationHandler; - } - - /** - * Trigger a container kill - */ - @Override - public void chaosAction() { - List liveContainers = - appState.cloneLiveContainerInfoList(); - // purge any and all components which declare that they are an AM - ListIterator containers = - liveContainers.listIterator(); - while (containers.hasNext()) { - RoleInstance instance = containers.next(); - if (SliderKeys.COMPONENT_AM.equals(instance.role)) { - containers.remove(); - } - } - int size = liveContainers.size(); - if (size > 0) { - int target = random.nextInt(size); - RoleInstance roleInstance = liveContainers.get(target); - log.info("Killing {}", roleInstance); - - queues.schedule(new ActionKillContainer(roleInstance.getContainerId(), - DELAY, TimeUnit.MILLISECONDS, operationHandler)); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java deleted file mode 100644 index 8948f0d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * 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.slider.server.appmaster.monkey; - -import com.codahale.metrics.MetricRegistry; -import org.apache.hadoop.service.AbstractService; -import org.apache.slider.api.InternalKeys; -import org.apache.slider.server.appmaster.actions.QueueAccess; -import org.apache.slider.server.appmaster.actions.RenewingAction; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; -import java.util.Random; -import java.util.concurrent.TimeUnit; - -/** - * A chaos monkey service which will invoke ChaosTarget events - */ -public class ChaosMonkeyService extends AbstractService { - protected static final Logger log = - LoggerFactory.getLogger(ChaosMonkeyService.class); - - private final MetricRegistry metrics; - private final QueueAccess queues; - private final Random random = new Random(); - - private final List chaosEntries = - new ArrayList(); - - public ChaosMonkeyService(MetricRegistry metrics, QueueAccess queues) { - super("ChaosMonkeyService"); - this.metrics = metrics; - this.queues = queues; - } - - /** - * Add a target ... it is only added if probability > 0 - * @param name name - * @param target chaos target - * @param probability probability - */ - public synchronized void addTarget(String name, - ChaosTarget target, long probability) { - if (probability > 0) { - log.info("Adding {} with probability {}", name, - ((double)probability) / InternalKeys.PROBABILITY_PERCENT_1); - chaosEntries.add(new ChaosEntry(name, target, probability, metrics)); - } else { - log.debug("Action {} not enabled", name); - } - } - - /** - * Get the number of targets in the list - * @return the count of added targets - */ - public int getTargetCount() { - return chaosEntries.size(); - } - - /** - * Iterate through all the entries and invoke chaos on those wanted - */ - public void play() { - for (ChaosEntry chaosEntry : chaosEntries) { - long p = randomPercentage(); - chaosEntry.maybeInvokeChaos(p); - } - } - - public int randomPercentage() { - return random.nextInt(InternalKeys.PROBABILITY_PERCENT_100); - } - - /** - * Check for callers to see if chaos should be triggered; shares the - * same random number source as the rest of the monkey entries - * @param probability probability - * @return true if the action should happen - */ - public boolean chaosCheck(long probability) { - return randomPercentage() < probability; - } - - /** - * Schedule the monkey - * - * @param delay initial delay - * @param timeUnit time unit - * @return true if it was scheduled (i.e. 1+ action) and interval > 0 - */ - public boolean schedule(long delay, long interval, TimeUnit timeUnit) { - if (interval > 0 && !chaosEntries.isEmpty()) { - queues.schedule(getChaosAction(delay, interval, timeUnit)); - return true; - } else { - return false; - } - } - - /** - * Get the chaos action - * - * @param delay - * @param timeUnit time unit - * @return the action to schedule - */ - public RenewingAction getChaosAction(long delay, - long interval, - TimeUnit timeUnit) { - RenewingAction action = new RenewingAction( - new MonkeyPlayAction(this, 0, TimeUnit.MILLISECONDS), - delay, - interval, - timeUnit, - 0 - ); - return action; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosTarget.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosTarget.java deleted file mode 100644 index 1c3a9ac..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosTarget.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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.slider.server.appmaster.monkey; - -public interface ChaosTarget { - - public void chaosAction(); -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/MonkeyPlayAction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/MonkeyPlayAction.java deleted file mode 100644 index 20e4466..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/MonkeyPlayAction.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.slider.server.appmaster.monkey; - -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.actions.AsyncAction; -import org.apache.slider.server.appmaster.actions.QueueAccess; -import org.apache.slider.server.appmaster.state.AppState; - -import java.util.concurrent.TimeUnit; - -/** - * Queueable action which calls {@link ChaosMonkeyService#play()} when - * executed. - */ -public class MonkeyPlayAction extends AsyncAction { - - private final ChaosMonkeyService monkey; - - public MonkeyPlayAction(ChaosMonkeyService monkey, long delay, - TimeUnit timeUnit) { - super("chaos monkey", delay, timeUnit); - this.monkey = monkey; - } - - @Override - public void execute(SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - monkey.play(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AbstractRMOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AbstractRMOperation.java deleted file mode 100644 index ed3f197..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AbstractRMOperation.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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.slider.server.appmaster.operations; - -public abstract class AbstractRMOperation { - - /** - * Execute the operation - * @param asyncRMClient client - * @param handler handler to perform the execution - */ - public abstract void execute(RMOperationHandlerActions handler); - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AsyncRMOperationHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AsyncRMOperationHandler.java deleted file mode 100644 index 7173354..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AsyncRMOperationHandler.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * 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.slider.server.appmaster.operations; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collection; -import java.util.List; - -/** - * Hands off RM operations to the Resource Manager. - */ -public class AsyncRMOperationHandler extends RMOperationHandler { - protected static final Logger log = - LoggerFactory.getLogger(AsyncRMOperationHandler.class); - private final AMRMClientAsync client; - private final Resource maxResources; - - public AsyncRMOperationHandler(AMRMClientAsync client, Resource maxResources) { - this.client = client; - this.maxResources = maxResources; - } - - @Override - public int cancelContainerRequests(Priority priority1, - Priority priority2, - int count) { - // need to revoke a previously issued container request - // so enum the sets and pick some - int remaining = cancelSinglePriorityRequests(priority1, count); - if (priority2 != null) { - remaining = cancelSinglePriorityRequests(priority2, remaining); - } - - return remaining; - } - - /** - * Cancel just one of the priority levels - * @param priority priority to cancel - * @param count count to cancel - * @return number of requests cancelled - */ - @SuppressWarnings("unchecked") - protected int cancelSinglePriorityRequests(Priority priority, - int count) { - List> requestSets = - client.getMatchingRequests(priority, "", maxResources); - if (count <= 0) { - return 0; - } - int remaining = count; - for (Collection requestSet : requestSets) { - if (remaining == 0) { - break; - } - for (AMRMClient.ContainerRequest request : requestSet) { - if (remaining == 0) { - break; - } - // a single release - cancelSingleRequest(request); - remaining --; - } - } - return remaining; - } - - @Override - @SuppressWarnings("unchecked") - public void cancelSingleRequest(AMRMClient.ContainerRequest request) { - // a single release - client.removeContainerRequest(request); - } - - @Override - public void releaseAssignedContainer(ContainerId containerId) { - log.debug("Releasing container {}", containerId); - - client.releaseAssignedContainer(containerId); - } - - @Override - @SuppressWarnings("unchecked") - public void addContainerRequest(AMRMClient.ContainerRequest req) { - client.addContainerRequest(req); - } - - @Override - public void updateBlacklist(List blacklistAdditions, - List blacklistRemovals) { - client.updateBlacklist(blacklistAdditions, blacklistRemovals); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/CancelSingleRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/CancelSingleRequest.java deleted file mode 100644 index d7673d3..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/CancelSingleRequest.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.slider.server.appmaster.operations; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.slider.server.appmaster.state.ContainerPriority; - -/** - * Cancel a container request - */ -public class CancelSingleRequest extends AbstractRMOperation { - - private final AMRMClient.ContainerRequest request; - - public CancelSingleRequest(AMRMClient.ContainerRequest request) { - Preconditions.checkArgument(request != null, "Null container request"); - this.request = request; - } - - @Override - public void execute(RMOperationHandlerActions handler) { - handler.cancelSingleRequest(request); - } - - public AMRMClient.ContainerRequest getRequest() { - return request; - } - - @Override - public String toString() { - return "Cancel container request" - + " for :" + ContainerPriority.toString(request.getPriority()) - + " request " + request; - } - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerReleaseOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerReleaseOperation.java deleted file mode 100644 index 4271d50..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerReleaseOperation.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.slider.server.appmaster.operations; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.slider.server.appmaster.state.ContainerPriority; - -public class ContainerReleaseOperation extends AbstractRMOperation { - - private final ContainerId containerId; - - public ContainerReleaseOperation(ContainerId containerId) { - Preconditions.checkArgument(containerId != null, "Null containerId"); - this.containerId = containerId; - } - - public ContainerId getContainerId() { - return containerId; - } - - @Override - public void execute(RMOperationHandlerActions handler) { - handler.releaseAssignedContainer(containerId); - } - - @Override - public String toString() { - return "release container " + containerId; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerRequestOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerRequestOperation.java deleted file mode 100644 index e29ddd0..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerRequestOperation.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.slider.server.appmaster.operations; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.slider.server.appmaster.state.ContainerPriority; - -/** - * A container request operation - */ -public class ContainerRequestOperation extends AbstractRMOperation { - - private final AMRMClient.ContainerRequest request; - - public ContainerRequestOperation(AMRMClient.ContainerRequest request) { - Preconditions.checkArgument(request != null, "Null container request"); - this.request = request; - } - - public AMRMClient.ContainerRequest getRequest() { - return request; - } - - public Priority getPriority() { - return request.getPriority(); - } - - public boolean getRelaxLocality() { - return request.getRelaxLocality(); - } - - @Override - public void execute(RMOperationHandlerActions handler) { - handler.addContainerRequest(request); - } - - @Override - public String toString() { - return "request container for role " - + ContainerPriority.toString(getPriority()) - + " request " + request - + " relaxLocality=" + getRelaxLocality(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandler.java deleted file mode 100644 index d0d038a..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandler.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.slider.server.appmaster.operations; - -import java.util.List; - -public abstract class RMOperationHandler implements RMOperationHandlerActions { - - @Override - public void execute(List operations) { - for (AbstractRMOperation operation : operations) { - operation.execute(this); - } - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandlerActions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandlerActions.java deleted file mode 100644 index bbaa933..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandlerActions.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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.slider.server.appmaster.operations; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.client.api.AMRMClient; - -import java.util.List; - -public interface RMOperationHandlerActions { - - /** - * Release an assigned container. - * @param containerId container - */ - void releaseAssignedContainer(ContainerId containerId); - - /** - * Issue a container request. - * @param request - */ - void addContainerRequest(AMRMClient.ContainerRequest request); - - /** - * Cancel a specific request. - * @param request request to cancel - */ - void cancelSingleRequest(AMRMClient.ContainerRequest request); - - /** - * Remove a container request. - * @param priority1 priority to remove at - * @param priority2 second priority to target - * @param count number to remove - */ - int cancelContainerRequests(Priority priority1, Priority priority2, int count); - - /** - * Blacklist resources. - * @param blacklistAdditions resources to add to the blacklist - * @param blacklistRemovals resources to remove from the blacklist - */ - void updateBlacklist(List blacklistAdditions, - List blacklistRemovals); - - /** - * Execute an entire list of operations. - * @param operations ops - */ - void execute(List operations); -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/UpdateBlacklistOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/UpdateBlacklistOperation.java deleted file mode 100644 index 90e2e5d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/UpdateBlacklistOperation.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.slider.server.appmaster.operations; - -import java.util.List; - -/** - * Update blacklisted resources for the application. - */ -public class UpdateBlacklistOperation extends AbstractRMOperation { - private final List blacklistAdditions; - private final List blacklistRemovals; - - public UpdateBlacklistOperation(List blacklistAdditions, - List blacklistRemovals) { - this.blacklistAdditions = blacklistAdditions; - this.blacklistRemovals = blacklistRemovals; - } - - @Override - public void execute(RMOperationHandlerActions handler) { - handler.updateBlacklist(blacklistAdditions, blacklistRemovals); - } - - @Override - public String toString() { - return "blacklist additions: " + blacklistAdditions - + ", blacklist removals: " + blacklistRemovals; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/RpcBinder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/RpcBinder.java deleted file mode 100644 index 2df9472..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/RpcBinder.java +++ /dev/null @@ -1,307 +0,0 @@ -/* - * 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.slider.server.appmaster.rpc; - -import com.google.common.base.Preconditions; -import com.google.protobuf.BlockingService; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.retry.RetryPolicies; -import org.apache.hadoop.io.retry.RetryPolicy; -import org.apache.hadoop.ipc.ProtobufRpcEngine; -import org.apache.hadoop.ipc.ProtocolProxy; -import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.ipc.RpcEngine; -import org.apache.hadoop.ipc.Server; -import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.SecretManager; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.yarn.api.ApplicationClientProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier; -import org.apache.hadoop.yarn.util.ConverterUtils; -import org.apache.slider.api.SliderClusterProtocol; -import org.apache.hadoop.yarn.service.conf.SliderExitCodes; -import org.apache.slider.common.tools.Duration; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadClusterStateException; -import org.apache.slider.core.exceptions.ErrorStrings; -import org.apache.slider.core.exceptions.SliderException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.security.PrivilegedExceptionAction; - -public class RpcBinder { - protected static final Logger log = - LoggerFactory.getLogger(RpcBinder.class); - - /** - * Create a protobuf server bonded to the specific socket address - * @param addr address to listen to; 0.0.0.0 as hostname acceptable - * @param conf config - * @param secretManager token secret handler - * @param numHandlers threads to service requests - * @param blockingService service to handle - * @param portRangeConfig range of ports - * @return the IPC server itself - * @throws IOException - */ - public static Server createProtobufServer(InetSocketAddress addr, - Configuration conf, - SecretManager secretManager, - int numHandlers, - BlockingService blockingService, - String portRangeConfig) throws - IOException { - Class sliderClusterAPIClass = registerSliderAPI( - conf); - RPC.Server server = new RPC.Builder(conf).setProtocol(sliderClusterAPIClass) - .setInstance(blockingService) - .setBindAddress(addr.getAddress() - .getCanonicalHostName()) - .setPort(addr.getPort()) - .setNumHandlers(numHandlers) - .setVerbose(false) - .setSecretManager(secretManager) - .setPortRangeConfig( - portRangeConfig) - .build(); - log.debug( - "Adding protocol " + sliderClusterAPIClass.getCanonicalName() + " to the server"); - server.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, sliderClusterAPIClass, - blockingService); - return server; - } - - /** - * Add the protobuf engine to the configuration. Harmless and inexpensive - * if repeated. - * @param conf configuration to patch - * @return the protocol class - */ - public static Class registerSliderAPI( - Configuration conf) { - Class sliderClusterAPIClass = - SliderClusterProtocolPB.class; - RPC.setProtocolEngine(conf, sliderClusterAPIClass, ProtobufRpcEngine.class); - - //quick sanity check here - assert verifyBondedToProtobuf(conf, sliderClusterAPIClass); - - return sliderClusterAPIClass; - } - - /** - * Verify that the conf is set up for protobuf transport of Slider RPC - * @param conf configuration - * @param sliderClusterAPIClass class for the API - * @return true if the RPC engine is protocol buffers - */ - public static boolean verifyBondedToProtobuf(Configuration conf, - Class sliderClusterAPIClass) { - return conf.getClass("rpc.engine." + sliderClusterAPIClass.getName(), - RpcEngine.class) .equals(ProtobufRpcEngine.class); - } - - - /** - * Connect to a server. May include setting up retry policies - * @param addr - * @param currentUser - * @param conf - * @param rpcTimeout - * @return - * @throws IOException - */ - public static SliderClusterProtocol connectToServer(InetSocketAddress addr, - UserGroupInformation currentUser, - Configuration conf, - int rpcTimeout) throws IOException { - Class sliderClusterAPIClass = - registerSliderAPI(conf); - - final RetryPolicy retryPolicy = RetryPolicies.TRY_ONCE_THEN_FAIL; - log.debug("Connecting to Slider AM at {}", addr); - ProtocolProxy protoProxy = - RPC.getProtocolProxy(sliderClusterAPIClass, - 1, - addr, - currentUser, - conf, - NetUtils.getDefaultSocketFactory(conf), - rpcTimeout, - retryPolicy); - SliderClusterProtocolPB endpoint = protoProxy.getProxy(); - return new SliderClusterProtocolProxy(endpoint, addr); - } - - - /** - * This loops for a limited period trying to get the Proxy - - * by doing so it handles AM failover - * @param conf configuration to patch and use - * @param rmClient client of the resource manager - * @param application application to work with - * @param connectTimeout timeout for the whole proxy operation to timeout - * (milliseconds). Use 0 to indicate "do not attempt to wait" -fail fast. - * @param rpcTimeout timeout for RPCs to block during communications - * @return the proxy - * @throws IOException IO problems - * @throws YarnException Slider-generated exceptions related to the binding - * failing. This can include the application finishing or timeouts - * @throws InterruptedException if a sleep operation waiting for - * the cluster to respond is interrupted. - */ - @SuppressWarnings("NestedAssignment") - public static SliderClusterProtocol getProxy(final Configuration conf, - final ApplicationClientProtocol rmClient, - ApplicationReport application, - final int connectTimeout, - final int rpcTimeout) - throws IOException, YarnException, InterruptedException { - ApplicationId appId; - appId = application.getApplicationId(); - Duration timeout = new Duration(connectTimeout); - timeout.start(); - Exception exception = null; - YarnApplicationState state = null; - try { - while (application != null && - (state = application.getYarnApplicationState()).equals( - YarnApplicationState.RUNNING)) { - - try { - return getProxy(conf, application, rpcTimeout); - } catch (IOException e) { - if (connectTimeout <= 0 || timeout.getLimitExceeded()) { - throw e; - } - exception = e; - } catch (YarnException e) { - if (connectTimeout <= 0 || timeout.getLimitExceeded()) { - throw e; - } - exception = e; - } - //at this point: app failed to work - log.debug("Could not connect to {}. Waiting for getting the latest AM address...", - appId); - Thread.sleep(1000); - //or get the app report - application = - rmClient.getApplicationReport( - GetApplicationReportRequest.newInstance(appId)).getApplicationReport(); - } - //get here if the app is no longer running. Raise a specific - //exception but init it with the previous failure - throw new BadClusterStateException( - exception, - ErrorStrings.E_FINISHED_APPLICATION, appId, state ); - } finally { - timeout.close(); - } - } - - /** - * Get a proxy from the application report - * @param conf config to use - * @param application app report - * @param rpcTimeout timeout in RPC operations - * @return the proxy - * @throws IOException - * @throws SliderException - * @throws InterruptedException - */ - public static SliderClusterProtocol getProxy(final Configuration conf, - final ApplicationReport application, - final int rpcTimeout) - throws IOException, SliderException, InterruptedException { - - String host = application.getHost(); - int port = application.getRpcPort(); - org.apache.hadoop.yarn.api.records.Token clientToAMToken = - application.getClientToAMToken(); - return createProxy(conf, host, port, clientToAMToken, rpcTimeout); - } - - /** - * - * @param conf config to use - * @param host hosname - * @param port port - * @param clientToAMToken auth token: only used in a secure cluster. - * converted via {@link ConverterUtils#convertFromYarn(org.apache.hadoop.yarn.api.records.Token, InetSocketAddress)} - * @param rpcTimeout timeout in RPC operations - * @return the proxy - * @throws SliderException - * @throws IOException - * @throws InterruptedException - */ - public static SliderClusterProtocol createProxy(final Configuration conf, - String host, - int port, - org.apache.hadoop.yarn.api.records.Token clientToAMToken, - final int rpcTimeout) throws - SliderException, - IOException, - InterruptedException { - String address = host + ":" + port; - if (SliderUtils.isUnset(host) || 0 == port) { - throw new SliderException(SliderExitCodes.EXIT_CONNECTIVITY_PROBLEM, - "Slider instance " - + " isn't providing a valid address for the" + - " Slider RPC protocol: " + address); - } - - UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); - final UserGroupInformation newUgi = UserGroupInformation.createRemoteUser( - currentUser.getUserName()); - final InetSocketAddress serviceAddr = - NetUtils.createSocketAddrForHost(host, port); - SliderClusterProtocol realProxy; - - log.debug("Connecting to {}", serviceAddr); - if (UserGroupInformation.isSecurityEnabled()) { - Preconditions.checkArgument(clientToAMToken != null, - "Null clientToAMToken"); - Token token = - ConverterUtils.convertFromYarn(clientToAMToken, serviceAddr); - newUgi.addToken(token); - realProxy = - newUgi.doAs(new PrivilegedExceptionAction() { - @Override - public SliderClusterProtocol run() throws IOException { - return connectToServer(serviceAddr, newUgi, conf, rpcTimeout); - } - }); - } else { - realProxy = connectToServer(serviceAddr, newUgi, conf, rpcTimeout); - } - return realProxy; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderAMPolicyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderAMPolicyProvider.java deleted file mode 100644 index 37c0a70..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderAMPolicyProvider.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * 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.slider.server.appmaster.rpc; - -import org.apache.hadoop.security.authorize.PolicyProvider; -import org.apache.hadoop.security.authorize.Service; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; - -/** - * {@link PolicyProvider} for Slider protocols. - */ - -public class SliderAMPolicyProvider extends PolicyProvider { - - private static final Service[] services = - new Service[] { - new Service(SliderXmlConfKeys.KEY_PROTOCOL_ACL, SliderClusterProtocolPB.class) - }; - - @SuppressWarnings("ReturnOfCollectionOrArrayField") - @Override - public Service[] getServices() { - return services; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPB.java deleted file mode 100644 index 7d237de..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPB.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.slider.server.appmaster.rpc; - -import org.apache.slider.api.SliderClusterProtocol; -import org.apache.slider.api.proto.SliderClusterAPI; - -public interface SliderClusterProtocolPB extends SliderClusterAPI.SliderClusterProtocolPB.BlockingInterface{ - - long versionID = SliderClusterProtocol.versionID; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPBImpl.java deleted file mode 100644 index 526ab7c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPBImpl.java +++ /dev/null @@ -1,225 +0,0 @@ -/* - * 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.slider.server.appmaster.rpc; - -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; -import org.apache.slider.api.SliderClusterProtocol; -import org.apache.slider.api.proto.Messages; - -import java.io.IOException; - -/** - * Server-side Relay from Protobuf to internal RPC. - * - */ -public class SliderClusterProtocolPBImpl implements SliderClusterProtocolPB { - - private SliderClusterProtocol real; - - public SliderClusterProtocolPBImpl(SliderClusterProtocol real) { - this.real = real; - } - - private ServiceException wrap(Exception e) { - if (e instanceof ServiceException) { - return (ServiceException) e; - } - return new ServiceException(e); - } - - public long getProtocolVersion(String protocol, long clientVersion) - throws IOException { - return SliderClusterProtocol.versionID; - } - - @Override - public Messages.StopClusterResponseProto stopCluster(RpcController controller, - Messages.StopClusterRequestProto request) throws ServiceException { - try { - return real.stopCluster(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.UpgradeContainersResponseProto upgradeContainers(RpcController controller, - Messages.UpgradeContainersRequestProto request) throws ServiceException { - try { - return real.upgradeContainers(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.FlexComponentsResponseProto flexComponents( - RpcController controller, Messages.FlexComponentsRequestProto request) - throws ServiceException { - try { - return real.flexComponents(request); - } catch (IOException e) { - throw wrap(e); - } - } - - @Override - public Messages.GetJSONClusterStatusResponseProto getJSONClusterStatus( - RpcController controller, - Messages.GetJSONClusterStatusRequestProto request) throws ServiceException { - try { - return real.getJSONClusterStatus(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.ListNodeUUIDsByRoleResponseProto listNodeUUIDsByRole( - RpcController controller, - Messages.ListNodeUUIDsByRoleRequestProto request) throws ServiceException { - try { - return real.listNodeUUIDsByRole(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.GetNodeResponseProto getNode(RpcController controller, - Messages.GetNodeRequestProto request) throws ServiceException { - try { - return real.getNode(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.GetClusterNodesResponseProto getClusterNodes(RpcController controller, - Messages.GetClusterNodesRequestProto request) throws ServiceException { - try { - return real.getClusterNodes(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.EchoResponseProto echo(RpcController controller, - Messages.EchoRequestProto request) throws ServiceException { - try { - return real.echo(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.KillContainerResponseProto killContainer(RpcController controller, - Messages.KillContainerRequestProto request) throws ServiceException { - try { - return real.killContainer(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.AMSuicideResponseProto amSuicide(RpcController controller, - Messages.AMSuicideRequestProto request) throws ServiceException { - try { - return real.amSuicide(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.ApplicationLivenessInformationProto getLivenessInformation( - RpcController controller, - Messages.GetApplicationLivenessRequestProto request) throws ServiceException { - try { - return real.getLivenessInformation(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.GetLiveContainersResponseProto getLiveContainers(RpcController controller, - Messages.GetLiveContainersRequestProto request) throws ServiceException { - try { - return real.getLiveContainers(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.ContainerInformationProto getLiveContainer(RpcController controller, - Messages.GetLiveContainerRequestProto request) throws ServiceException { - try { - return real.getLiveContainer(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.GetLiveComponentsResponseProto getLiveComponents(RpcController controller, - Messages.GetLiveComponentsRequestProto request) throws ServiceException { - try { - return real.getLiveComponents(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.ComponentInformationProto getLiveComponent(RpcController controller, - Messages.GetLiveComponentRequestProto request) throws ServiceException { - try { - return real.getLiveComponent(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.GetLiveNodesResponseProto getLiveNodes(RpcController controller, - Messages.GetLiveNodesRequestProto request) throws ServiceException { - try { - return real.getLiveNodes(request); - } catch (Exception e) { - throw wrap(e); - } - } - - @Override - public Messages.NodeInformationProto getLiveNode(RpcController controller, - Messages.GetLiveNodeRequestProto request) throws ServiceException { - try { - return real.getLiveNode(request); - } catch (Exception e) { - throw wrap(e); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolProxy.java deleted file mode 100644 index 2e40a9b..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolProxy.java +++ /dev/null @@ -1,270 +0,0 @@ -/* - * 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.slider.server.appmaster.rpc; - -import com.google.common.base.Preconditions; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; -import org.apache.hadoop.ipc.ProtobufHelper; -import org.apache.hadoop.ipc.ProtocolSignature; -import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.slider.api.SliderClusterProtocol; -import org.apache.slider.api.proto.Messages; - -import java.io.IOException; -import java.net.InetSocketAddress; - -public class SliderClusterProtocolProxy implements SliderClusterProtocol { - - private static final RpcController NULL_CONTROLLER = null; - private final SliderClusterProtocolPB endpoint; - private final InetSocketAddress address; - - public SliderClusterProtocolProxy(SliderClusterProtocolPB endpoint, - InetSocketAddress address) { - Preconditions.checkArgument(endpoint != null, "null endpoint"); - Preconditions.checkNotNull(address != null, "null address"); - this.endpoint = endpoint; - this.address = address; - } - - @Override - public String toString() { - final StringBuilder sb = - new StringBuilder("SliderClusterProtocolProxy{"); - sb.append("address=").append(address); - sb.append('}'); - return sb.toString(); - } - - @Override - public ProtocolSignature getProtocolSignature(String protocol, - long clientVersion, - int clientMethodsHash) - throws IOException { - if (!protocol.equals(RPC.getProtocolName(SliderClusterProtocolPB.class))) { - throw new IOException("Serverside implements " + - RPC.getProtocolName(SliderClusterProtocolPB.class) + - ". The following requested protocol is unknown: " + - protocol); - } - - return ProtocolSignature.getProtocolSignature(clientMethodsHash, - RPC.getProtocolVersion( - SliderClusterProtocol.class), - SliderClusterProtocol.class); - } - - @Override - public long getProtocolVersion(String protocol, long clientVersion) - throws IOException { - return SliderClusterProtocol.versionID; - } - - private IOException convert(ServiceException se) { - IOException ioe = ProtobufHelper.getRemoteException(se); - if (ioe instanceof RemoteException) { - RemoteException remoteException = (RemoteException) ioe; - return remoteException.unwrapRemoteException(); - } - return ioe; - } - - @Override public Messages.StopClusterResponseProto stopCluster( - Messages.StopClusterRequestProto request) - throws IOException, YarnException { - try { - return endpoint.stopCluster(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.UpgradeContainersResponseProto upgradeContainers( - Messages.UpgradeContainersRequestProto request) throws IOException, - YarnException { - try { - return endpoint.upgradeContainers(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.FlexComponentsResponseProto flexComponents( - Messages.FlexComponentsRequestProto request) throws IOException { - try { - return endpoint.flexComponents(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.GetJSONClusterStatusResponseProto getJSONClusterStatus( - Messages.GetJSONClusterStatusRequestProto request) throws - IOException, - YarnException { - try { - return endpoint.getJSONClusterStatus(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.ListNodeUUIDsByRoleResponseProto listNodeUUIDsByRole(Messages.ListNodeUUIDsByRoleRequestProto request) throws - IOException, - YarnException { - try { - return endpoint.listNodeUUIDsByRole(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.GetNodeResponseProto getNode(Messages.GetNodeRequestProto request) throws - IOException, - YarnException { - try { - return endpoint.getNode(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.GetClusterNodesResponseProto getClusterNodes(Messages.GetClusterNodesRequestProto request) throws - IOException, - YarnException { - try { - return endpoint.getClusterNodes(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - - @Override - public Messages.EchoResponseProto echo(Messages.EchoRequestProto request) throws - IOException, - YarnException { - try { - return endpoint.echo(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - - @Override - public Messages.KillContainerResponseProto killContainer(Messages.KillContainerRequestProto request) throws - IOException, - YarnException { - try { - return endpoint.killContainer(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.AMSuicideResponseProto amSuicide(Messages.AMSuicideRequestProto request) throws - IOException { - try { - return endpoint.amSuicide(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.ApplicationLivenessInformationProto getLivenessInformation( - Messages.GetApplicationLivenessRequestProto request) throws IOException { - try { - return endpoint.getLivenessInformation(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.GetLiveContainersResponseProto getLiveContainers(Messages.GetLiveContainersRequestProto request) throws - IOException { - try { - return endpoint.getLiveContainers(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.ContainerInformationProto getLiveContainer(Messages.GetLiveContainerRequestProto request) throws - IOException { - try { - return endpoint.getLiveContainer(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.GetLiveComponentsResponseProto getLiveComponents(Messages.GetLiveComponentsRequestProto request) throws - IOException { - try { - return endpoint.getLiveComponents(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.ComponentInformationProto getLiveComponent(Messages.GetLiveComponentRequestProto request) throws - IOException { - try { - return endpoint.getLiveComponent(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.GetLiveNodesResponseProto getLiveNodes(Messages.GetLiveNodesRequestProto request) - throws IOException { - try { - return endpoint.getLiveNodes(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } - - @Override - public Messages.NodeInformationProto getLiveNode(Messages.GetLiveNodeRequestProto request) - throws IOException { - try { - return endpoint.getLiveNode(NULL_CONTROLLER, request); - } catch (ServiceException e) { - throw convert(e); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderIPCService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderIPCService.java deleted file mode 100644 index 22f9bc3..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderIPCService.java +++ /dev/null @@ -1,406 +0,0 @@ -/* - * 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.slider.server.appmaster.rpc; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.ipc.ProtocolSignature; -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import org.apache.slider.api.SliderClusterProtocol; -import org.apache.slider.api.proto.Messages; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.types.ApplicationLivenessInformation; -import org.apache.slider.api.types.ComponentInformation; -import org.apache.slider.api.types.ContainerInformation; -import org.apache.slider.api.types.NodeInformation; -import org.apache.slider.api.types.NodeInformationList; -import org.apache.slider.core.exceptions.ServiceNotReadyException; -import org.apache.slider.core.main.LauncherExitCodes; -import org.apache.slider.core.persist.JsonSerDeser; -import org.apache.slider.server.appmaster.AppMasterActionOperations; -import org.apache.slider.server.appmaster.actions.ActionFlexCluster; -import org.apache.slider.server.appmaster.actions.ActionHalt; -import org.apache.slider.server.appmaster.actions.ActionKillContainer; -import org.apache.slider.server.appmaster.actions.ActionStopSlider; -import org.apache.slider.server.appmaster.actions.ActionUpgradeContainers; -import org.apache.slider.server.appmaster.actions.AsyncAction; -import org.apache.slider.server.appmaster.actions.QueueAccess; -import org.apache.slider.server.appmaster.management.MetricsAndMonitoring; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.state.StateAccessForProviders; -import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import static org.apache.slider.api.types.RestTypeMarshalling.marshall; -import static org.apache.slider.server.appmaster.web.rest.RestPaths.*; - -/** - * Implement the {@link SliderClusterProtocol}. - */ -@SuppressWarnings("unchecked") - -public class SliderIPCService extends AbstractService - implements SliderClusterProtocol { - - protected static final Logger log = - LoggerFactory.getLogger(SliderIPCService.class); - - private final QueueAccess actionQueues; - private final StateAccessForProviders state; - private final MetricsAndMonitoring metricsAndMonitoring; - private final AppMasterActionOperations amOperations; - private final ContentCache cache; - private static final JsonSerDeser jsonSerDeser = - new JsonSerDeser(Application.class); - - - /** - * This is the prefix used for metrics - */ - public static final String METRICS_PREFIX = - "org.apache.slider.api.SliderIPCService."; - - /** - * Constructor - * @param amOperations access to any AM operations - * @param state state view - * @param actionQueues queues for actions - * @param metricsAndMonitoring metrics - * @param cache - */ - public SliderIPCService(AppMasterActionOperations amOperations, - StateAccessForProviders state, QueueAccess actionQueues, - MetricsAndMonitoring metricsAndMonitoring, ContentCache cache) { - super("SliderIPCService"); - Preconditions.checkArgument(amOperations != null, "null amOperations"); - Preconditions.checkArgument(state != null, "null appState"); - Preconditions.checkArgument(actionQueues != null, "null actionQueues"); - Preconditions.checkArgument(metricsAndMonitoring != null, - "null metricsAndMonitoring"); - Preconditions.checkArgument(cache != null, "null cache"); - this.state = state; - this.actionQueues = actionQueues; - this.metricsAndMonitoring = metricsAndMonitoring; - this.amOperations = amOperations; - this.cache = cache; - } - - @Override //SliderClusterProtocol - public ProtocolSignature getProtocolSignature(String protocol, - long clientVersion, - int clientMethodsHash) throws IOException { - return ProtocolSignature.getProtocolSignature( - this, protocol, clientVersion, clientMethodsHash); - } - - - @Override //SliderClusterProtocol - public long getProtocolVersion(String protocol, long clientVersion) - throws IOException { - return SliderClusterProtocol.versionID; - } - - /** - * General actions to perform on a slider RPC call coming in - * @param operation operation to log - * @throws IOException problems - * @throws ServiceNotReadyException if the RPC service is constructed - * but not fully initialized - */ - protected void onRpcCall(String operation) throws IOException { - log.debug("Received call to {}", operation); - metricsAndMonitoring.markMeterAndCounter(METRICS_PREFIX + operation); - } - - /** - * Schedule an action - * @param action for delayed execution - */ - public void schedule(AsyncAction action) { - actionQueues.schedule(action); - } - - /** - * Queue an action for immediate execution in the executor thread - * @param action action to execute - */ - public void queue(AsyncAction action) { - actionQueues.put(action); - } - - @Override //SliderClusterProtocol - public Messages.StopClusterResponseProto stopCluster(Messages.StopClusterRequestProto request) - throws IOException, YarnException { - onRpcCall("stop"); - String message = request.getMessage(); - if (message == null) { - message = "application stopped by client"; - } - ActionStopSlider stopSlider = - new ActionStopSlider(message, - 1000, TimeUnit.MILLISECONDS, - LauncherExitCodes.EXIT_SUCCESS, - FinalApplicationStatus.SUCCEEDED, - message); - log.info("SliderAppMasterApi.stopCluster: {}", stopSlider); - schedule(stopSlider); - return Messages.StopClusterResponseProto.getDefaultInstance(); - } - - @Override //SliderClusterProtocol - public Messages.UpgradeContainersResponseProto upgradeContainers( - Messages.UpgradeContainersRequestProto request) throws IOException, - YarnException { - onRpcCall("upgrade"); - String message = request.getMessage(); - if (message == null) { - message = "application containers upgraded by client"; - } - ActionUpgradeContainers upgradeContainers = - new ActionUpgradeContainers( - "Upgrade containers", - 1000, TimeUnit.MILLISECONDS, - LauncherExitCodes.EXIT_SUCCESS, - FinalApplicationStatus.SUCCEEDED, - request.getContainerList(), - request.getComponentList(), - message); - log.info("SliderAppMasterApi.upgradeContainers: {}", upgradeContainers); - schedule(upgradeContainers); - return Messages.UpgradeContainersResponseProto.getDefaultInstance(); - } - - @Override - public Messages.FlexComponentsResponseProto flexComponents( - Messages.FlexComponentsRequestProto request) throws IOException { - onRpcCall("flex"); - schedule(new ActionFlexCluster("flex", 1, TimeUnit.MILLISECONDS, request)); - return Messages.FlexComponentsResponseProto.newBuilder().build(); - } - - @Override //SliderClusterProtocol - public Messages.GetJSONClusterStatusResponseProto getJSONClusterStatus( - Messages.GetJSONClusterStatusRequestProto request) - throws IOException, YarnException { - onRpcCall("getstatus"); - String result; - //quick update - //query and json-ify - Application application = state.refreshClusterStatus(); - String stat = jsonSerDeser.toJson(application); - return Messages.GetJSONClusterStatusResponseProto.newBuilder() - .setClusterSpec(stat).build(); - } - - @Override //SliderClusterProtocol - public Messages.ListNodeUUIDsByRoleResponseProto listNodeUUIDsByRole(Messages.ListNodeUUIDsByRoleRequestProto request) - throws IOException, YarnException { - onRpcCall("listnodes)"); - String role = request.getRole(); - Messages.ListNodeUUIDsByRoleResponseProto.Builder builder = - Messages.ListNodeUUIDsByRoleResponseProto.newBuilder(); - List nodes = state.enumLiveInstancesInRole(role); - for (RoleInstance node : nodes) { - builder.addUuid(node.id); - } - return builder.build(); - } - - @Override //SliderClusterProtocol - public Messages.GetNodeResponseProto getNode(Messages.GetNodeRequestProto request) - throws IOException, YarnException { - onRpcCall("getnode"); - RoleInstance instance = state.getLiveInstanceByContainerID( - request.getUuid()); - return Messages.GetNodeResponseProto.newBuilder() - .setClusterNode(instance.toProtobuf()) - .build(); - } - - @Override //SliderClusterProtocol - public Messages.GetClusterNodesResponseProto getClusterNodes( - Messages.GetClusterNodesRequestProto request) - throws IOException, YarnException { - onRpcCall("getclusternodes"); - List - clusterNodes = state.getLiveInstancesByContainerIDs( - request.getUuidList()); - - Messages.GetClusterNodesResponseProto.Builder builder = - Messages.GetClusterNodesResponseProto.newBuilder(); - for (RoleInstance node : clusterNodes) { - builder.addClusterNode(node.toProtobuf()); - } - //at this point: a possibly empty list of nodes - return builder.build(); - } - - @Override - public Messages.EchoResponseProto echo(Messages.EchoRequestProto request) - throws IOException, YarnException { - onRpcCall("echo"); - Messages.EchoResponseProto.Builder builder = - Messages.EchoResponseProto.newBuilder(); - String text = request.getText(); - log.info("Echo request size ={}", text.length()); - log.info(text); - //now return it - builder.setText(text); - return builder.build(); - } - - @Override - public Messages.KillContainerResponseProto killContainer(Messages.KillContainerRequestProto request) - throws IOException, YarnException { - onRpcCall("killcontainer"); - String containerID = request.getId(); - log.info("Kill Container {}", containerID); - //throws NoSuchNodeException if it is missing - RoleInstance instance = - state.getLiveInstanceByContainerID(containerID); - queue(new ActionKillContainer(instance.getContainerId(), 0, TimeUnit.MILLISECONDS, - amOperations)); - Messages.KillContainerResponseProto.Builder builder = - Messages.KillContainerResponseProto.newBuilder(); - builder.setSuccess(true); - return builder.build(); - } - - - @Override - public Messages.AMSuicideResponseProto amSuicide( - Messages.AMSuicideRequestProto request) - throws IOException { - onRpcCall("amsuicide"); - int signal = request.getSignal(); - String text = request.getText(); - if (text == null) { - text = ""; - } - int delay = request.getDelay(); - log.info("AM Suicide with signal {}, message {} delay = {}", signal, text, - delay); - ActionHalt action = new ActionHalt(signal, text, delay, - TimeUnit.MILLISECONDS); - schedule(action); - return Messages.AMSuicideResponseProto.getDefaultInstance(); - } - - @Override - public Messages.ApplicationLivenessInformationProto getLivenessInformation( - Messages.GetApplicationLivenessRequestProto request) throws IOException { - ApplicationLivenessInformation info = - state.getApplicationLivenessInformation(); - return marshall(info); - } - - @Override - public Messages.GetLiveContainersResponseProto getLiveContainers( - Messages.GetLiveContainersRequestProto request) - throws IOException { - Map infoMap = - (Map) cache.lookupWithIOE(LIVE_CONTAINERS); - Messages.GetLiveContainersResponseProto.Builder builder = - Messages.GetLiveContainersResponseProto.newBuilder(); - - for (Map.Entry entry : infoMap.entrySet()) { - builder.addNames(entry.getKey()); - builder.addContainers(marshall(entry.getValue())); - } - return builder.build(); - } - - @Override - public Messages.ContainerInformationProto getLiveContainer(Messages.GetLiveContainerRequestProto request) - throws IOException { - String containerId = request.getContainerId(); - RoleInstance id = state.getLiveInstanceByContainerID(containerId); - ContainerInformation containerInformation = id.serialize(); - return marshall(containerInformation); - } - - @Override - public Messages.GetLiveComponentsResponseProto getLiveComponents(Messages.GetLiveComponentsRequestProto request) - throws IOException { - Map infoMap = - (Map) cache.lookupWithIOE(LIVE_COMPONENTS); - Messages.GetLiveComponentsResponseProto.Builder builder = - Messages.GetLiveComponentsResponseProto.newBuilder(); - - for (Map.Entry entry : infoMap.entrySet()) { - builder.addNames(entry.getKey()); - builder.addComponents(marshall(entry.getValue())); - } - return builder.build(); - } - - - @Override - public Messages.ComponentInformationProto getLiveComponent(Messages.GetLiveComponentRequestProto request) - throws IOException { - String name = request.getName(); - try { - return marshall(state.getComponentInformation(name)); - } catch (YarnRuntimeException e) { - throw new FileNotFoundException("Unknown component: " + name); - } - } - - @Override - public Messages.GetLiveNodesResponseProto getLiveNodes(Messages.GetLiveNodesRequestProto request) - throws IOException { - NodeInformationList info = (NodeInformationList) cache.lookupWithIOE(LIVE_NODES); - Messages.GetLiveNodesResponseProto.Builder builder = - Messages.GetLiveNodesResponseProto.newBuilder(); - - for (NodeInformation nodeInformation : info) { - builder.addNodes(marshall(nodeInformation)); - } - return builder.build(); - } - - - @Override - public Messages.NodeInformationProto getLiveNode(Messages.GetLiveNodeRequestProto request) - throws IOException { - String name = request.getName(); - NodeInformation nodeInformation = state.getNodeInformation(name); - if (nodeInformation != null) { - return marshall(nodeInformation); - } else { - throw new FileNotFoundException("Unknown host: " + name); - } - } - - private Messages.WrappedJsonProto wrap(String json) { - Messages.WrappedJsonProto.Builder builder = - Messages.WrappedJsonProto.newBuilder(); - builder.setJson(json); - return builder.build(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderRPCSecurityInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderRPCSecurityInfo.java deleted file mode 100644 index 5b127b8..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderRPCSecurityInfo.java +++ /dev/null @@ -1,87 +0,0 @@ -/** -* 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.slider.server.appmaster.rpc; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.KerberosInfo; -import org.apache.hadoop.security.SecurityInfo; -import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.security.token.TokenInfo; -import org.apache.hadoop.security.token.TokenSelector; -import org.apache.hadoop.yarn.security.client.ClientToAMTokenSelector; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; - -import java.lang.annotation.Annotation; - -/** - * This is where security information goes. - * It is referred to in the META-INF/services/org.apache.hadoop.security.SecurityInfo - * resource of this JAR, which is used to find the binding info - */ -public class SliderRPCSecurityInfo extends SecurityInfo { - - @Override - public KerberosInfo getKerberosInfo(Class protocol, Configuration conf) { - if (!protocol.equals(SliderClusterProtocolPB.class)) { - return null; - } - return new KerberosInfo() { - - @Override - public Class annotationType() { - return null; - } - - @Override - public String serverPrincipal() { - return SliderXmlConfKeys.KEY_KERBEROS_PRINCIPAL; - } - - @Override - public String clientPrincipal() { - return null; - } - }; - } - - @Override - public TokenInfo getTokenInfo(Class protocol, Configuration conf) { - if (!protocol.equals(SliderClusterProtocolPB.class)) { - return null; - } - return new TokenInfo() { - - @Override - public Class annotationType() { - return null; - } - - @Override - public Class> - value() { - return ClientToAMTokenSelector.class; - } - - @Override - public String toString() { - return "SliderClusterProtocolPB token info"; - } - }; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/security/SecurityConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/security/SecurityConfiguration.java deleted file mode 100644 index 75eccd0..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/security/SecurityConfiguration.java +++ /dev/null @@ -1,161 +0,0 @@ -/* - * 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.slider.server.appmaster.security; - -import com.google.common.base.Preconditions; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.UserGroupInformation; -import static org.apache.slider.core.main.LauncherExitCodes.EXIT_UNAUTHORIZED; - -import org.apache.slider.api.resource.Application; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.SliderException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; - -/** - * Class keeping code security information - */ -public class SecurityConfiguration { - - protected static final Logger log = - LoggerFactory.getLogger(SecurityConfiguration.class); - private final Configuration configuration; - private final Application application; - private String clusterName; - - public SecurityConfiguration(Configuration configuration, - Application application, - String clusterName) throws SliderException { - Preconditions.checkNotNull(configuration); - Preconditions.checkNotNull(application); - Preconditions.checkNotNull(clusterName); - this.configuration = configuration; - this.application = application; - this.clusterName = clusterName; - validate(); - } - - private void validate() throws SliderException { - if (isSecurityEnabled()) { - // TODO use AM configuration rather than app configuration - String principal = application.getConfiguration().getProperty( - SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL); - if(SliderUtils.isUnset(principal)) { - // if no login identity is available, fail - UserGroupInformation loginUser = null; - try { - loginUser = getLoginUser(); - } catch (IOException e) { - throw new SliderException(EXIT_UNAUTHORIZED, e, - "No principal configured for the application and " - + "exception raised during retrieval of login user. " - + "Unable to proceed with application " - + "initialization. Please ensure a value " - + "for %s exists in the application " - + "configuration or the login issue is addressed", - SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL); - } - if (loginUser == null) { - throw new SliderException(EXIT_UNAUTHORIZED, - "No principal configured for the application " - + "and no login user found. " - + "Unable to proceed with application " - + "initialization. Please ensure a value " - + "for %s exists in the application " - + "configuration or the login issue is addressed", - SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL); - } - } - // ensure that either local or distributed keytab mechanism is enabled, - // but not both - String keytabFullPath = application.getConfiguration().getProperty( - SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH); - String keytabName = application.getConfiguration().getProperty( - SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME); - if (SliderUtils.isSet(keytabFullPath) && SliderUtils.isSet(keytabName)) { - throw new SliderException(EXIT_UNAUTHORIZED, - "Both a keytab on the cluster host (%s) and a" - + " keytab to be retrieved from HDFS (%s) are" - + " specified. Please configure only one keytab" - + " retrieval mechanism.", - SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH, - SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME); - - } - } - } - - protected UserGroupInformation getLoginUser() throws IOException { - return UserGroupInformation.getLoginUser(); - } - - public boolean isSecurityEnabled() { - return SliderUtils.isHadoopClusterSecure(configuration); - } - - public String getPrincipal() throws IOException { - String principal = application.getConfiguration().getProperty( - SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL); - if (SliderUtils.isUnset(principal)) { - principal = UserGroupInformation.getLoginUser().getShortUserName(); - log.info("No principal set in the slider configuration. Will use AM " + - "login identity {} to attempt keytab-based login", principal); - } - - return principal; - } - - public boolean isKeytabProvided() { - String keytabLocalPath = application.getConfiguration().getProperty( - SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH); - String keytabName = application.getConfiguration().getProperty( - SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME); - return StringUtils.isNotBlank(keytabLocalPath) - || StringUtils.isNotBlank(keytabName); - - } - - public File getKeytabFile() - throws SliderException, IOException { - //TODO implement this for dash semantic - String keytabFullPath = application.getConfiguration().getProperty( - SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH); - File localKeytabFile; - if (SliderUtils.isUnset(keytabFullPath)) { - // get the keytab - String keytabName = application.getConfiguration().getProperty( - SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME); - log.info("No host keytab file path specified. Will attempt to retrieve" - + " keytab file {} as a local resource for the container", - keytabName); - // download keytab to local, protected directory - localKeytabFile = new File(SliderKeys.KEYTAB_DIR, keytabName); - } else { - log.info("Using host keytab file {} for login", keytabFullPath); - localKeytabFile = new File(keytabFullPath); - } - return localKeytabFile; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AbstractClusterServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AbstractClusterServices.java deleted file mode 100644 index 54f384b..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AbstractClusterServices.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; - -/** - * Cluster services offered by the YARN infrastructure. - */ -public abstract class AbstractClusterServices { - - private final DefaultResourceCalculator - defaultResourceCalculator = new DefaultResourceCalculator(); - - /** - * Create a resource for requests - * @return a resource which can be built up. - */ - public abstract Resource newResource(); - - public abstract Resource newResource(int memory, int cores); - - /** - * Normalise memory, CPU and other resources according to the YARN AM-supplied - * values and the resource calculator in use (currently hard-coded to the - * {@link DefaultResourceCalculator}. - * Those resources which aren't normalized (currently: CPU) are left - * as is. - * @param resource resource requirements of a role - * @param minR minimum values of this queue - * @param maxR max values of this queue - * @return a normalized value. - */ - public Resource normalize(Resource resource, Resource minR, Resource maxR) { - Preconditions.checkArgument(resource != null, "null resource"); - Preconditions.checkArgument(minR != null, "null minR"); - Preconditions.checkArgument(maxR != null, "null maxR"); - - Resource normalize = defaultResourceCalculator.normalize(resource, minR, - maxR, minR); - return newResource(normalize.getMemory(), resource.getVirtualCores()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java deleted file mode 100644 index ba923bc..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java +++ /dev/null @@ -1,2120 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -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.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.metrics2.lib.MutableGaugeInt; -import org.apache.hadoop.security.UserGroupInformation; -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.NodeId; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import org.apache.hadoop.yarn.util.resource.Resources; -import org.apache.slider.api.ClusterNode; -import org.apache.slider.api.InternalKeys; -import org.apache.slider.api.ServiceApiConstants; -import org.apache.slider.api.StatusKeys; -import org.apache.slider.api.proto.Messages; -import org.apache.slider.api.proto.Messages.ComponentCountProto; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.ApplicationState; -import org.apache.slider.api.resource.Component; -import org.apache.slider.api.resource.ConfigFile; -import org.apache.slider.api.types.ApplicationLivenessInformation; -import org.apache.slider.api.types.ComponentInformation; -import org.apache.slider.api.types.RoleStatistics; -import org.apache.hadoop.yarn.service.conf.SliderExitCodes; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadClusterStateException; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.core.exceptions.ErrorStrings; -import org.apache.slider.core.exceptions.NoSuchNodeException; -import org.apache.slider.core.exceptions.SliderInternalStateException; -import org.apache.slider.core.exceptions.TriggerClusterTeardownException; -import org.apache.slider.core.zk.ZKIntegration; -import org.apache.slider.providers.PlacementPolicy; -import org.apache.slider.providers.ProviderRole; -import org.apache.slider.server.appmaster.management.MetricsAndMonitoring; -import org.apache.slider.server.appmaster.management.MetricsConstants; -import org.apache.hadoop.yarn.service.metrics.ServiceMetrics; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.ContainerReleaseOperation; -import org.apache.slider.server.appmaster.operations.ContainerRequestOperation; -import org.apache.slider.server.appmaster.operations.UpdateBlacklistOperation; -import org.apache.hadoop.yarn.service.timelineservice.ServiceTimelinePublisher; -import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.net.URI; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.ListIterator; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY; -import static org.apache.hadoop.registry.client.api.RegistryConstants.DEFAULT_REGISTRY_ZK_QUORUM; -import static org.apache.hadoop.registry.client.api.RegistryConstants.KEY_DNS_DOMAIN; -import static org.apache.hadoop.registry.client.api.RegistryConstants.KEY_REGISTRY_ZK_QUORUM; -import static org.apache.slider.api.ResourceKeys.*; -import static org.apache.slider.api.ServiceApiConstants.*; -import static org.apache.slider.api.StateValues.*; -import static org.apache.slider.api.resource.ApplicationState.STARTED; - -/** - * The model of all the ongoing state of a Slider AM. - * - * concurrency rules: any method which begins with build - * is not synchronized and intended to be used during - * initialization. - */ -public class AppState { - protected static final Logger log = - LoggerFactory.getLogger(AppState.class); - - private final AbstractClusterServices recordFactory; - - private final MetricsAndMonitoring metricsAndMonitoring; - /** - * Flag set to indicate the application is live -this only happens - * after the buildInstance operation - */ - private boolean applicationLive = false; - - private Application app; - - // priority_id -> RoleStatus - private final Map roleStatusMap = - new ConcurrentSkipListMap<>(); - - // component_name -> ProviderRole - private final Map roles = - new ConcurrentHashMap<>(); - - private final ConcurrentSkipListMap rolePriorityMap = - new ConcurrentSkipListMap<>(); - - /** - * Hash map of the containers we have. This includes things that have - * been allocated but are not live; it is a superset of the live list - */ - private final ConcurrentMap ownedContainers = - new ConcurrentHashMap<>(); - - /** - * Hash map of the containers we have released, but we - * are still awaiting acknowledgements on. Any failure of these - * containers is treated as a successful outcome - */ - private final ConcurrentMap containersBeingReleased = - new ConcurrentHashMap<>(); - - /** - * Map of requested nodes. This records the command used to start it, - * resources, etc. When container started callback is received, - * the node is promoted from here to the containerMap - */ - private final Map startingContainers = - new ConcurrentHashMap<>(); - - /** - * List of completed nodes. This isn't kept in the CD as it gets too - * big for the RPC responses. Indeed, we should think about how deep to get this - */ - private final Map completedContainers - = new ConcurrentHashMap<>(); - - /** - * Nodes that failed to start. - * Again, kept out of the CD - */ - private final Map failedContainers = - new ConcurrentHashMap<>(); - - /** - * Nodes that came assigned to a role above that - * which were asked for -this appears to happen - */ - private final Set surplusContainers = new HashSet<>(); - - /** - * Map of containerID to cluster nodes, for status reports. - * Access to this should be synchronized on the clusterDescription - */ - private final Map liveNodes = - new ConcurrentHashMap<>(); - private final AtomicInteger completionOfNodeNotInLiveListEvent = - new AtomicInteger(); - private final AtomicInteger completionOfUnknownContainerEvent = - new AtomicInteger(); - - /** - * limits of container core numbers in this queue - */ - private int containerMaxCores; - private int containerMinCores; - - /** - * limits of container memory in this queue - */ - private int containerMaxMemory; - private int containerMinMemory; - - private RoleHistory roleHistory; - private long startTimeThreshold; - - private int failureThreshold = 10; - private int nodeFailureThreshold = 3; - - private String logServerURL = ""; - public Map globalTokens = new HashMap<>(); - /** - * Selector of containers to release; application wide. - */ - private ContainerReleaseSelector containerReleaseSelector; - private Resource minResource; - private Resource maxResource; - - private ServiceMetrics appMetrics; - - private ServiceTimelinePublisher serviceTimelinePublisher; - - // A cache for loading config files from remote such as hdfs - public LoadingCache configFileCache = null; - - /** - * Create an instance - * @param recordFactory factory for YARN records - * @param metricsAndMonitoring metrics and monitoring services - */ - public AppState(AbstractClusterServices recordFactory, - MetricsAndMonitoring metricsAndMonitoring) { - Preconditions.checkArgument(recordFactory != null, "null recordFactory"); - Preconditions.checkArgument(metricsAndMonitoring != null, "null metricsAndMonitoring"); - this.recordFactory = recordFactory; - this.metricsAndMonitoring = metricsAndMonitoring; - } - - - public Map getRoleStatusMap() { - return roleStatusMap; - } - - protected Map getRoleMap() { - return roles; - } - - public Map getRolePriorityMap() { - return rolePriorityMap; - } - - private Map getStartingContainers() { - return startingContainers; - } - - private Map getCompletedContainers() { - return completedContainers; - } - - public Map getFailedContainers() { - return failedContainers; - } - - public Map getLiveContainers() { - return liveNodes; - } - - /** - * Get the current view of the cluster status. - * This is read-only - * to the extent that changes here do not trigger updates in the - * application state. - * @return the cluster status - */ - public synchronized Application getClusterStatus() { - return app; - } - - /** - * Get the role history of the application - * @return the role history - */ - @VisibleForTesting - public RoleHistory getRoleHistory() { - return roleHistory; - } - - @VisibleForTesting - public void setRoleHistory(RoleHistory roleHistory) { - this.roleHistory = roleHistory; - } - - /** - * Get the path used for history files - * @return the directory used for history files - */ - @VisibleForTesting - public Path getHistoryPath() { - return roleHistory.getHistoryPath(); - } - - /** - * Set the container limits -the min and max values for - * resource requests. All requests must be multiples of the min - * values. - * @param minMemory min memory MB - * @param maxMemory maximum memory - * @param minCores min v core count - * @param maxCores maximum cores - */ - public void setContainerLimits(int minMemory,int maxMemory, int minCores, int maxCores) { - containerMinCores = minCores; - containerMaxCores = maxCores; - containerMinMemory = minMemory; - containerMaxMemory = maxMemory; - minResource = recordFactory.newResource(containerMinMemory, containerMinCores); - maxResource = recordFactory.newResource(containerMaxMemory, containerMaxCores); - } - - public boolean isApplicationLive() { - return applicationLive; - } - - - public synchronized void buildInstance(AppStateBindingInfo binding) - throws BadClusterStateException, BadConfigException, IOException { - binding.validate(); - containerReleaseSelector = binding.releaseSelector; - - // set the cluster specification (once its dependency the client properties - // is out the way - this.app = binding.application; - appMetrics = ServiceMetrics.register(app.getName(), - "Metrics for service"); - appMetrics.tag("type", "Metrics type [component or service]", "service"); - appMetrics.tag("appId", "Application id for service", app.getId()); - - org.apache.slider.api.resource.Configuration conf = app.getConfiguration(); - startTimeThreshold = - conf.getPropertyLong(InternalKeys.INTERNAL_CONTAINER_FAILURE_SHORTLIFE, - InternalKeys.DEFAULT_INTERNAL_CONTAINER_FAILURE_SHORTLIFE); - failureThreshold = conf.getPropertyInt(CONTAINER_FAILURE_THRESHOLD, - DEFAULT_CONTAINER_FAILURE_THRESHOLD); - nodeFailureThreshold = conf.getPropertyInt(NODE_FAILURE_THRESHOLD, - DEFAULT_NODE_FAILURE_THRESHOLD); - initGlobalTokensForSubstitute(binding); - - // build the initial component list - Collection sortedComponents = ServiceApiUtil - .sortByDependencies(app.getComponents()); - int priority = 1; - for (Component component : sortedComponents) { - priority = getNewPriority(priority); - String name = component.getName(); - if (roles.containsKey(name)) { - continue; - } - log.info("Adding component: " + name); - createComponent(name, component, priority++); - } - - //then pick up the requirements -// buildRoleRequirementsFromResources(); - - // set up the role history - roleHistory = new RoleHistory(roleStatusMap.values(), recordFactory); - roleHistory.onStart(binding.fs, binding.historyPath); - // trigger first node update - roleHistory.onNodesUpdated(binding.nodeReports); - //rebuild any live containers - rebuildModelFromRestart(binding.liveContainers); - - // any am config options to pick up - logServerURL = binding.serviceConfig.get(YarnConfiguration.YARN_LOG_SERVER_URL, ""); - //mark as live - applicationLive = true; - app.setState(STARTED); - createConfigFileCache(binding.fs); - } - - private void initGlobalTokensForSubstitute(AppStateBindingInfo binding) - throws IOException { - // ZK - globalTokens.put(ServiceApiConstants.CLUSTER_ZK_QUORUM, - binding.serviceConfig - .getTrimmed(KEY_REGISTRY_ZK_QUORUM, DEFAULT_REGISTRY_ZK_QUORUM)); - String user = UserGroupInformation.getCurrentUser().getShortUserName(); - globalTokens - .put(SERVICE_ZK_PATH, ZKIntegration.mkClusterPath(user, app.getName())); - - globalTokens.put(ServiceApiConstants.USER, user); - String dnsDomain = binding.serviceConfig.getTrimmed(KEY_DNS_DOMAIN); - if (dnsDomain != null && !dnsDomain.isEmpty()) { - globalTokens.put(ServiceApiConstants.DOMAIN, dnsDomain); - } - // HDFS - String clusterFs = binding.serviceConfig.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, binding.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; - } - } - }); - } - - public ProviderRole createComponent(String name, Component component, - int priority) throws BadConfigException { - org.apache.slider.api.resource.Configuration conf = - component.getConfiguration(); - long placementTimeout = conf.getPropertyLong(PLACEMENT_ESCALATE_DELAY, - DEFAULT_PLACEMENT_ESCALATE_DELAY_SECONDS); - long placementPolicy = conf.getPropertyLong(COMPONENT_PLACEMENT_POLICY, - PlacementPolicy.DEFAULT); - int threshold = conf.getPropertyInt(NODE_FAILURE_THRESHOLD, - nodeFailureThreshold); - String label = conf.getProperty(YARN_LABEL_EXPRESSION, - DEF_YARN_LABEL_EXPRESSION); - ProviderRole newRole = - new ProviderRole(name, priority, (int)placementPolicy, threshold, - placementTimeout, label, component); - buildRole(newRole, component); - log.info("Created a new role " + newRole); - return newRole; - } - - @VisibleForTesting - public synchronized void updateComponents(Map - componentCounts) throws BadConfigException { - for (Component component : app.getComponents()) { - if (componentCounts.containsKey(component.getName())) { - long count = componentCounts.get(component.getName()); - component.setNumberOfContainers(count); - ProviderRole role = roles.get(component.getName()); - if (role != null && roleStatusMap.get(role.id) != null) { - setDesiredContainers(roleStatusMap.get(role.id), (int) count); - } - } - } - } - - public synchronized void updateComponents( - Messages.FlexComponentsRequestProto requestProto) - throws BadConfigException { - Map componentCounts = new HashMap<>(); - for (ComponentCountProto componentCount : requestProto - .getComponentsList()) { - componentCounts.put(componentCount.getName(), componentCount - .getNumberOfContainers()); - } - updateComponents(componentCounts); - } - - /** - * build the role requirements from the cluster specification - * @return a list of any dynamically added provider roles - */ - -// private List buildRoleRequirementsFromResources() -// throws BadConfigException { -// -// List newRoles = new ArrayList<>(0); -// -// // now update every role's desired count. -// // if there are no instance values, that role count goes to zero -// // Add all the existing roles -// // component name -> number of containers -// Map groupCounts = new HashMap<>(); -// -// for (RoleStatus roleStatus : getRoleStatusMap().values()) { -// if (roleStatus.isExcludeFromFlexing()) { -// // skip inflexible roles, e.g AM itself -// continue; -// } -// long currentDesired = roleStatus.getDesired(); -// String role = roleStatus.getName(); -// String roleGroup = roleStatus.getGroup(); -// Component component = roleStatus.getProviderRole().component; -// int desiredInstanceCount = component.getNumberOfContainers().intValue(); -// -// int newDesired = desiredInstanceCount; -// if (component.getUniqueComponentSupport()) { -// Integer groupCount = 0; -// if (groupCounts.containsKey(roleGroup)) { -// groupCount = groupCounts.get(roleGroup); -// } -// -// newDesired = desiredInstanceCount - groupCount; -// -// if (newDesired > 0) { -// newDesired = 1; -// groupCounts.put(roleGroup, groupCount + newDesired); -// } else { -// newDesired = 0; -// } -// } -// -// if (newDesired == 0) { -// log.info("Role {} has 0 instances specified", role); -// } -// if (currentDesired != newDesired) { -// log.info("Role {} flexed from {} to {}", role, currentDesired, -// newDesired); -// setDesiredContainers(roleStatus, newDesired); -// } -// } -// -// log.info("Counts per component: " + groupCounts); -// // now the dynamic ones. Iterate through the the cluster spec and -// // add any role status entries not in the role status -// -// List list = new ArrayList<>(getRoleStatusMap().values()); -// for (RoleStatus roleStatus : list) { -// String name = roleStatus.getName(); -// Component component = roleStatus.getProviderRole().component; -// if (roles.containsKey(name)) { -// continue; -// } -// if (component.getUniqueComponentSupport()) { -// // THIS NAME IS A GROUP -// int desiredInstanceCount = component.getNumberOfContainers().intValue(); -// Integer groupCount = 0; -// if (groupCounts.containsKey(name)) { -// groupCount = groupCounts.get(name); -// } -// log.info("Component " + component.getName() + ", current count = " -// + groupCount + ", desired count = " + desiredInstanceCount); -// for (int i = groupCount + 1; i <= desiredInstanceCount; i++) { -// int priority = roleStatus.getPriority(); -// // this is a new instance of an existing group -// String newName = String.format("%s%d", name, i); -// int newPriority = getNewPriority(priority + i - 1); -// log.info("Adding new role {}", newName); -// ProviderRole dynamicRole = -// createComponent(newName, name, component, newPriority); -// RoleStatus newRole = buildRole(dynamicRole); -// incDesiredContainers(newRole); -// log.info("New role {}", newRole); -// if (roleHistory != null) { -// roleHistory.addNewRole(newRole); -// } -// newRoles.add(dynamicRole); -// } -// } else { -// // this is a new value -// log.info("Adding new role {}", name); -// ProviderRole dynamicRole = -// createComponent(name, name, component, roleStatus.getPriority()); -// RoleStatus newRole = buildRole(dynamicRole); -// incDesiredContainers(roleStatus, -// component.getNumberOfContainers().intValue()); -// log.info("New role {}", newRole); -// if (roleHistory != null) { -// roleHistory.addNewRole(newRole); -// } -// newRoles.add(dynamicRole); -// } -// } -// // and fill in all those roles with their requirements -// buildRoleResourceRequirements(); -// -// return newRoles; -// } - - private int getNewPriority(int start) { - if (!rolePriorityMap.containsKey(start)) { - return start; - } - return rolePriorityMap.lastKey() + 1; - } - - /** - * Add knowledge of a role. - * This is a build-time operation that is not synchronized, and - * should be used while setting up the system state -before servicing - * requests. - * @param providerRole role to add - * @return the role status built up - * @throws BadConfigException if a role of that priority already exists - */ - public RoleStatus buildRole(ProviderRole providerRole, Component component) - throws BadConfigException { - // build role status map - int priority = providerRole.id; - if (roleStatusMap.containsKey(priority)) { - throw new BadConfigException("Duplicate component priority Key: %s and %s", - providerRole, roleStatusMap.get(priority)); - } - RoleStatus roleStatus = new RoleStatus(providerRole); - roleStatus.setResourceRequirements(buildResourceRequirements(roleStatus)); - long prev = roleStatus.getDesired(); - setDesiredContainers(roleStatus, component.getNumberOfContainers().intValue()); - log.info("Set desired containers for component " + component.getName() + - " from " + prev + " to " + roleStatus.getDesired()); - roleStatusMap.put(priority, roleStatus); - String name = providerRole.name; - roles.put(name, providerRole); - rolePriorityMap.put(priority, providerRole); - // register its entries - metricsAndMonitoring.addMetricSet(MetricsConstants.PREFIX_SLIDER_ROLES + name, roleStatus); - return roleStatus; - } - - /** - * Look up the status entry of a role or raise an exception - * @param key role ID - * @return the status entry - * @throws RuntimeException if the role cannot be found - */ - public RoleStatus lookupRoleStatus(int key) { - RoleStatus rs = getRoleStatusMap().get(key); - if (rs == null) { - throw new RuntimeException("Cannot find role for role ID " + key); - } - return rs; - } - - /** - * Look up the status entry of a container or raise an exception - * - * @param c container - * @return the status entry - * @throws RuntimeException if the role cannot be found - */ - public RoleStatus lookupRoleStatus(Container c) { - return lookupRoleStatus(ContainerPriority.extractRole(c)); - } - - - /** - * Look up a role in the map - * @param name role name - * @return the instance - * @throws YarnRuntimeException if not found - */ - public RoleStatus lookupRoleStatus(String name) throws YarnRuntimeException { - ProviderRole providerRole = roles.get(name); - if (providerRole == null) { - throw new YarnRuntimeException("Unknown role " + name); - } - return lookupRoleStatus(providerRole.id); - } - - - /** - * Clone the list of active (==owned) containers - * @return the list of role instances representing all owned containers - */ - public synchronized List cloneOwnedContainerList() { - Collection values = ownedContainers.values(); - return new ArrayList<>(values); - } - - /** - * Get the number of active (==owned) containers - * @return - */ - public int getNumOwnedContainers() { - return ownedContainers.size(); - } - - /** - * Look up an active container: any container that the AM has, even - * if it is not currently running/live - */ - public RoleInstance getOwnedContainer(ContainerId id) { - return ownedContainers.get(id); - } - - /** - * Remove an owned container - * @param id container ID - * @return the instance removed - */ - private RoleInstance removeOwnedContainer(ContainerId id) { - return ownedContainers.remove(id); - } - - /** - * set/update an owned container - * @param id container ID - * @param instance - * @return - */ - private RoleInstance putOwnedContainer(ContainerId id, - RoleInstance instance) { - return ownedContainers.put(id, instance); - } - - /** - * Clone the live container list. This is synchronized. - * @return a snapshot of the live node list - */ - public synchronized List cloneLiveContainerInfoList() { - List allRoleInstances; - Collection values = getLiveContainers().values(); - allRoleInstances = new ArrayList<>(values); - return allRoleInstances; - } - - /** - * Lookup live instance by string value of container ID - * @param containerId container ID as a string - * @return the role instance for that container - * @throws NoSuchNodeException if it does not exist - */ - public synchronized RoleInstance getLiveInstanceByContainerID(String containerId) - throws NoSuchNodeException { - Collection nodes = getLiveContainers().values(); - return findNodeInCollection(containerId, nodes); - } - - /** - * Lookup owned instance by string value of container ID - * @param containerId container ID as a string - * @return the role instance for that container - * @throws NoSuchNodeException if it does not exist - */ - public synchronized RoleInstance getOwnedInstanceByContainerID(String containerId) - throws NoSuchNodeException { - Collection nodes = ownedContainers.values(); - return findNodeInCollection(containerId, nodes); - } - - /** - * Iterate through a collection of role instances to find one with a - * specific (string) container ID - * @param containerId container ID as a string - * @param nodes collection - * @return the found node - * @throws NoSuchNodeException if there was no match - */ - private RoleInstance findNodeInCollection(String containerId, - Collection nodes) throws NoSuchNodeException { - RoleInstance found = null; - for (RoleInstance node : nodes) { - if (containerId.equals(node.id)) { - found = node; - break; - } - } - if (found != null) { - return found; - } else { - //at this point: no node - throw new NoSuchNodeException("Unknown node: " + containerId); - } - } - - public synchronized List getLiveInstancesByContainerIDs( - Collection containerIDs) { - //first, a hashmap of those containerIDs is built up - Set uuidSet = new HashSet(containerIDs); - List nodes = new ArrayList(uuidSet.size()); - Collection clusterNodes = getLiveContainers().values(); - - for (RoleInstance node : clusterNodes) { - if (uuidSet.contains(node.id)) { - nodes.add(node); - } - } - //at this point: a possibly empty list of nodes - return nodes; - } - - /** - * Enum all nodes by role. - * @param role role, or "" for all roles - * @return a list of nodes, may be empty - */ - public synchronized List enumLiveNodesInRole(String role) { - List nodes = new ArrayList(); - Collection allRoleInstances = getLiveContainers().values(); - for (RoleInstance node : allRoleInstances) { - if (role.isEmpty() || role.equals(node.role)) { - nodes.add(node); - } - } - return nodes; - } - - - /** - * enum nodes by role ID, from either the owned or live node list - * @param roleId role the container must be in - * @param owned flag to indicate "use owned list" rather than the smaller - * "live" list - * @return a list of nodes, may be empty - */ - public synchronized List enumNodesWithRoleId(int roleId, - boolean owned) { - List nodes = new ArrayList(); - Collection allRoleInstances; - allRoleInstances = owned ? ownedContainers.values() : liveNodes.values(); - for (RoleInstance node : allRoleInstances) { - if (node.roleId == roleId) { - nodes.add(node); - } - } - return nodes; - } - - /** - * Build an instance map. - * @return the map of Role name to list of role instances - */ - private synchronized Map> createRoleToInstanceMap() { - Map> map = new HashMap>(); - for (RoleInstance node : getLiveContainers().values()) { - List containers = map.get(node.role); - if (containers == null) { - containers = new ArrayList(); - map.put(node.role, containers); - } - containers.add(node.id); - } - return map; - } - - /** - * Build a map of Component_name -> ContainerId -> ClusterNode - * - * @return the map of Role name to list of Cluster Nodes - */ - public synchronized Map> createRoleToClusterNodeMap() { - Map> map = new HashMap<>(); - for (RoleInstance node : getLiveContainers().values()) { - - Map containers = map.get(node.role); - if (containers == null) { - containers = new HashMap(); - map.put(node.role, containers); - } - ClusterNode clusterNode = node.toClusterNode(); - containers.put(clusterNode.name, clusterNode); - } - return map; - } - - /** - * Notification called just before the NM is asked to - * start a container - * @param container container to start - * @param instance clusterNode structure - */ - public void containerStartSubmitted(Container container, - RoleInstance instance) { - instance.state = STATE_SUBMITTED; - instance.container = container; - instance.createTime = now(); - getStartingContainers().put(container.getId(), instance); - putOwnedContainer(container.getId(), instance); - roleHistory.onContainerStartSubmitted(container, instance); - } - - /** - * Note that a container has been submitted for release; update internal state - * and mark the associated ContainerInfo released field to indicate that - * while it is still in the active list, it has been queued for release. - * - * @param container container - * @throws SliderInternalStateException if there is no container of that ID - * on the active list - */ - public synchronized void containerReleaseSubmitted(Container container) - throws SliderInternalStateException { - ContainerId id = container.getId(); - //look up the container - RoleInstance instance = getOwnedContainer(id); - if (instance == null) { - throw new SliderInternalStateException( - "No active container with ID " + id); - } - //verify that it isn't already released - if (containersBeingReleased.containsKey(id)) { - throw new SliderInternalStateException( - "Container %s already queued for release", id); - } - instance.released = true; - containersBeingReleased.put(id, instance.container); - roleHistory.onContainerReleaseSubmitted(container); - } - - /** - * Create a container request. - * Update internal state, such as the role request count. - * Anti-Affine: the {@link RoleStatus#outstandingAArequest} is set here. - * This is where role history information will be used for placement decisions. - * @param role role - * @return the container request to submit or null if there is none - */ - private AMRMClient.ContainerRequest createContainerRequest(RoleStatus role) { - if (role.isAntiAffinePlacement()) { - return createAAContainerRequest(role); - } else { - OutstandingRequest request = roleHistory.requestContainerForRole(role); - if (request != null) { - return request.getIssuedRequest(); - } else { - return null; - } - } - } - - /** - * Create a container request. - * Update internal state, such as the role request count. - * Anti-Affine: the {@link RoleStatus#outstandingAArequest} is set here. - * This is where role history information will be used for placement decisions. - * @param role role - * @return the container request to submit or null if there is none - */ - private AMRMClient.ContainerRequest createAAContainerRequest(RoleStatus role) { - OutstandingRequest request = roleHistory.requestContainerForAARole(role); - if (request == null) { - return null; - } - role.setOutstandingAArequest(request); - return request.getIssuedRequest(); - } - - @VisibleForTesting - public void incRequestedContainers(RoleStatus role) { - log.info("Incrementing requested containers for {}", role.getName()); - role.getComponentMetrics().containersRequested.incr(); - appMetrics.containersRequested.incr(); - } - - private void decRequestedContainers(RoleStatus role) { - role.getComponentMetrics().containersRequested.decr(); - appMetrics.containersRequested.decr(); - log.info("Decrementing requested containers for {} by {} to {}", role - .getName(), 1, role.getComponentMetrics().containersRequested.value()); - } - - private int decRequestedContainersToFloor(RoleStatus role, int delta) { - int actual = decMetricToFloor(role.getComponentMetrics() - .containersRequested, delta); - appMetrics.containersRequested.decr(actual); - log.info("Decrementing requested containers for {} by {} to {}", role - .getName(), actual, role.getComponentMetrics().containersRequested - .value()); - return actual; - } - - private int decAAPendingToFloor(RoleStatus role, int delta) { - int actual = decMetricToFloor(role.getComponentMetrics() - .pendingAAContainers, delta); - appMetrics.pendingAAContainers.decr(actual); - log.info("Decrementing AA pending containers for {} by {} to {}", role - .getName(), actual, role.getComponentMetrics().pendingAAContainers - .value()); - return actual; - } - - private int decMetricToFloor(MutableGaugeInt metric, int delta) { - int currentValue = metric.value(); - int decrAmount = delta; - if (currentValue - delta < 0) { - decrAmount = currentValue; - } - metric.decr(decrAmount); - return decrAmount; - } - - @VisibleForTesting - public void incRunningContainers(RoleStatus role) { - role.getComponentMetrics().containersRunning.incr(); - appMetrics.containersRunning.incr(); - } - - private void decRunningContainers(RoleStatus role) { - role.getComponentMetrics().containersRunning.decr(); - appMetrics.containersRunning.decr(); - } - - private void setDesiredContainers(RoleStatus role, int n) { - int delta = n - role.getComponentMetrics().containersDesired.value(); - role.getComponentMetrics().containersDesired.set(n); - appMetrics.containersDesired.incr(delta); - } - - private void incCompletedContainers(RoleStatus role) { - role.getComponentMetrics().containersSucceeded.incr(); - appMetrics.containersSucceeded.incr(); - } - - @VisibleForTesting - public void incFailedContainers(RoleStatus role, ContainerOutcome outcome) { - switch (outcome) { - case Preempted: - appMetrics.containersPreempted.incr(); - role.getComponentMetrics().containersPreempted.incr(); - break; - case Disk_failure: - appMetrics.containersDiskFailure.incr(); - appMetrics.containersFailed.incr(); - role.getComponentMetrics().containersDiskFailure.incr(); - role.getComponentMetrics().containersFailed.incr(); - break; -// case Failed: -// appMetrics.failedSinceLastThreshold.incr(); -// appMetrics.containersFailed.incr(); -// role.getComponentMetrics().failedSinceLastThreshold.incr(); -// role.getComponentMetrics().containersFailed.incr(); -// break; -// case Failed_limits_exceeded: -// appMetrics.containersLimitsExceeded.incr(); -// appMetrics.failedSinceLastThreshold.incr(); -// appMetrics.containersFailed.incr(); -// role.getComponentMetrics().containersLimitsExceeded.incr(); -// role.getComponentMetrics().failedSinceLastThreshold.incr(); -// role.getComponentMetrics().containersFailed.incr(); -// break; -// default: -// appMetrics.failedSinceLastThreshold.incr(); -// appMetrics.containersFailed.incr(); -// role.getComponentMetrics().failedSinceLastThreshold.incr(); -// role.getComponentMetrics().containersFailed.incr(); -// break; - } - } - - /** - * Build up the resource requirements for this role from the cluster - * specification, including substituting max allowed values if the - * specification asked for it (except when - * {@link org.apache.slider.api.ResourceKeys#YARN_RESOURCE_NORMALIZATION_ENABLED} - * is set to false). - * @param role role - * during normalization - */ - public Resource buildResourceRequirements(RoleStatus role) { - // Set up resource requirements from role values - String name = role.getName(); - Component component = role.getProviderRole().component; - int cores = DEF_YARN_CORES; - if (component.getResource() != null && component.getResource().getCpus() - != null) { - cores = Math.min(containerMaxCores, component.getResource().getCpus()); - } - if (cores <= 0) { - cores = DEF_YARN_CORES; - } - long rawMem = DEF_YARN_MEMORY; - if (component.getResource() != null && component.getResource().getMemory() - != null) { - if (YARN_RESOURCE_MAX.equals(component.getResource().getMemory())) { - rawMem = containerMaxMemory; - } else { - rawMem = Long.parseLong(component.getResource().getMemory()); - } - } - boolean normalize = component.getConfiguration().getPropertyBool( - YARN_RESOURCE_NORMALIZATION_ENABLED, true); - if (!normalize) { - log.info("Resource normalization: disabled"); - log.debug("Component {} has RAM={}, vCores={}", name, rawMem, cores); - return Resources.createResource(rawMem, cores); - } - long mem = Math.min(containerMaxMemory, rawMem); - if (mem <= 0) { - mem = DEF_YARN_MEMORY; - } - Resource capability = Resource.newInstance(mem, cores); - log.debug("Component {} has RAM={}, vCores={}", name, mem, cores); - Resource normalized = recordFactory.normalize(capability, minResource, - maxResource); - if (!Resources.equals(normalized, capability)) { - // resource requirements normalized to something other than asked for. - // LOG @ WARN so users can see why this is happening. - log.warn("Resource requirements of {} normalized" + - " from {} to {}", name, capability, normalized); - } - return normalized; - } - - /** - * add a launched container to the node map for status responses - * @param container id - * @param node node details - */ - private void addLaunchedContainer(Container container, RoleInstance node) { - node.container = container; - if (node.role == null) { - throw new RuntimeException( - "Unknown role for node " + node); - } - getLiveContainers().put(node.getContainerId(), node); - //tell role history - roleHistory.onContainerStarted(container); - } - - /** - * container start event - * @param containerId container that is to be started - * @return the role instance, or null if there was a problem - */ - public synchronized RoleInstance onNodeManagerContainerStarted(ContainerId containerId) { - try { - return innerOnNodeManagerContainerStarted(containerId); - } catch (YarnRuntimeException e) { - log.error("NodeManager callback on started container {} failed", - containerId, - e); - return null; - } - } - - /** - * container start event handler -throwing an exception on problems - * @param containerId container that is to be started - * @return the role instance - * @throws RuntimeException on problems - */ - @VisibleForTesting - public RoleInstance innerOnNodeManagerContainerStarted(ContainerId containerId) { - RoleInstance instance = getOwnedContainer(containerId); - if (instance == null) { - //serious problem - throw new YarnRuntimeException("Container not in active containers start "+ - containerId); - } - if (instance.role == null) { - throw new YarnRuntimeException("Component instance has no instance name " + - instance); - } - instance.startTime = now(); - RoleInstance starting = getStartingContainers().remove(containerId); - if (null == starting) { - throw new YarnRuntimeException( - "Container "+ containerId +" is already started"); - } - instance.state = STATE_LIVE; - Container container = instance.container; - addLaunchedContainer(container, instance); - return instance; - } - - /** - * update the application state after a failure to start a container. - * This is perhaps where blacklisting could be most useful: failure - * to start a container is a sign of a more serious problem - * than a later exit. - * - * -relayed from NMClientAsync.CallbackHandler - * @param containerId failing container - * @param thrown what was thrown - */ - public synchronized void onNodeManagerContainerStartFailed(ContainerId containerId, - Throwable thrown) { - removeOwnedContainer(containerId); - RoleInstance instance = getStartingContainers().remove(containerId); - if (null != instance) { - RoleStatus roleStatus = lookupRoleStatus(instance.roleId); - String text; - if (null != thrown) { - text = SliderUtils.stringify(thrown); - } else { - text = "container start failure"; - } - instance.diagnostics = text; - roleStatus.noteFailed(text); - getFailedContainers().put(containerId, instance); - roleHistory.onNodeManagerContainerStartFailed(instance.container); - incFailedContainers(roleStatus, ContainerOutcome.Failed); - } - } - - /** - * Handle node update from the RM. This syncs up the node map with the RM's view - * @param updatedNodes updated nodes - */ - public synchronized NodeUpdatedOutcome onNodesUpdated(List updatedNodes) { - boolean changed = roleHistory.onNodesUpdated(updatedNodes); - if (changed) { - log.info("YARN cluster changed —cancelling current AA requests"); - List operations = cancelOutstandingAARequests(); - log.debug("Created {} cancel requests", operations.size()); - return new NodeUpdatedOutcome(true, operations); - } - return new NodeUpdatedOutcome(false, new ArrayList<>(0)); - } - - /** - * Return value of the {@link #onNodesUpdated(List)} call. - */ - public static class NodeUpdatedOutcome { - public final boolean clusterChanged; - public final List operations; - - public NodeUpdatedOutcome(boolean clusterChanged, - List operations) { - this.clusterChanged = clusterChanged; - this.operations = operations; - } - } - /** - * Is a role short lived by the threshold set for this application - * @param instance instance - * @return true if the instance is considered short lived - */ - @VisibleForTesting - public boolean isShortLived(RoleInstance instance) { - long time = now(); - long started = instance.startTime; - boolean shortlived; - if (started > 0) { - long duration = time - started; - shortlived = duration < (startTimeThreshold * 1000); - log.info("Duration {} and startTimeThreshold {}", duration, startTimeThreshold); - } else { - // never even saw a start event - shortlived = true; - } - return shortlived; - } - - /** - * Current time in milliseconds. Made protected for - * the option to override it in tests. - * @return the current time. - */ - protected long now() { - return System.currentTimeMillis(); - } - - /** - * This is a very small class to send a multiple result back from - * the completion operation - */ - public static class NodeCompletionResult { - public boolean surplusNode = false; - public RoleInstance roleInstance; - // did the container fail for *any* reason? - public boolean containerFailed = false; - // detailed outcome on the container failure - public ContainerOutcome outcome = ContainerOutcome.Completed; - public int exitStatus = 0; - public boolean unknownNode = false; - - public String toString() { - final StringBuilder sb = - new StringBuilder("NodeCompletionResult{"); - sb.append("surplusNode=").append(surplusNode); - sb.append(", roleInstance=").append(roleInstance); - sb.append(", exitStatus=").append(exitStatus); - sb.append(", containerFailed=").append(containerFailed); - sb.append(", outcome=").append(outcome); - sb.append(", unknownNode=").append(unknownNode); - sb.append('}'); - return sb.toString(); - } - } - - /** - * handle completed node in the CD -move something from the live - * server list to the completed server list. - * @param status the node that has just completed - * @return NodeCompletionResult - */ - public synchronized NodeCompletionResult onCompletedContainer( - ContainerStatus status) { - ContainerId containerId = status.getContainerId(); - NodeCompletionResult result = new NodeCompletionResult(); - RoleInstance roleInstance; - - int exitStatus = status.getExitStatus(); - result.exitStatus = exitStatus; - if (containersBeingReleased.containsKey(containerId)) { - log.info("Container was queued for release : {}", containerId); - Container container = containersBeingReleased.remove(containerId); - RoleStatus roleStatus = lookupRoleStatus(container); - decRunningContainers(roleStatus); - incCompletedContainers(roleStatus); - log.info("decrementing role count for role {} to {}; completed={}", - roleStatus.getName(), - roleStatus.getComponentMetrics().containersRunning.value(), - roleStatus.getComponentMetrics().containersSucceeded.value()); - result.outcome = ContainerOutcome.Completed; - roleHistory.onReleaseCompleted(container); - - } else if (surplusContainers.remove(containerId)) { - //its a surplus one being purged - result.surplusNode = true; - } else { - // a container has failed or been killed - // use the exit code to determine the outcome - result.containerFailed = true; - result.outcome = ContainerOutcome.fromExitStatus(exitStatus); - - roleInstance = removeOwnedContainer(containerId); - if (roleInstance != null) { - RoleStatus roleStatus = lookupRoleStatus(roleInstance.roleId); - incFailedContainers(roleStatus, result.outcome); - failedContainers.put(containerId, roleInstance); - } else { - // the container may have been noted as failed already, so look - // it up - roleInstance = failedContainers.get(containerId); - } - if (roleInstance != null) { - int roleId = roleInstance.roleId; - String rolename = roleInstance.role; - log.info("Failed container in role[{}] : {}", roleId, - roleInstance.getCompInstanceName()); - try { - RoleStatus roleStatus = lookupRoleStatus(roleInstance.roleId); - decRunningContainers(roleStatus); - roleStatus.getProviderRole().failedInstances.offer(roleInstance); - boolean shortLived = isShortLived(roleInstance); - String message; - Container failedContainer = roleInstance.container; - - //build the failure message - if (failedContainer != null) { - String completedLogsUrl = getLogsURLForContainer(failedContainer); - message = String.format("Failure %s on host %s (%d): %s", - roleInstance.getContainerId(), - failedContainer.getNodeId().getHost(), - exitStatus, - completedLogsUrl); - } else { - message = String.format("Failure %s (%d)", containerId, exitStatus); - } - roleStatus.noteFailed(message); - long failed = - roleStatus.getComponentMetrics().containersFailed.value(); - log.info("Current count of failed role[{}] {} = {}", - roleId, rolename, failed); - if (failedContainer != null) { - roleHistory.onFailedContainer(failedContainer, shortLived, result.outcome); - } - - } catch (YarnRuntimeException e1) { - log.error("Failed container of unknown role {}", roleId); - } - } else { - //this isn't a known container. - - log.error("Notified of completed container {} that is not in the list" + - " of active or failed containers", containerId); - completionOfUnknownContainerEvent.incrementAndGet(); - result.unknownNode = true; - } - } - - if (result.surplusNode) { - //a surplus node - return result; - } - - //record the complete node's details; this pulls it from the livenode set - //remove the node - ContainerId id = status.getContainerId(); - log.info("Removing node ID {}", id); - RoleInstance node = getLiveContainers().remove(id); - if (node != null) { - node.state = STATE_DESTROYED; - node.exitCode = exitStatus; - node.diagnostics = status.getDiagnostics(); - getCompletedContainers().put(id, node); - result.roleInstance = node; - } else { - // not in the list - log.warn("Received notification of completion of unknown node {}", id); - completionOfNodeNotInLiveListEvent.incrementAndGet(); - } - - // and the active node list if present - removeOwnedContainer(containerId); - - // finally, verify the node doesn't exist any more - assert !containersBeingReleased.containsKey( - containerId) : "container still in release queue"; - assert !getLiveContainers().containsKey( - containerId) : " container still in live nodes"; - assert getOwnedContainer(containerId) == - null : "Container still in active container list"; - - return result; - } - - /** - * Get the URL log for a container - * @param c container - * @return the URL or "" if it cannot be determined - */ - protected String getLogsURLForContainer(Container c) { - if (c==null) { - return null; - } - String user = null; - try { - user = SliderUtils.getCurrentUser().getShortUserName(); - } catch (IOException ignored) { - } - String completedLogsUrl = ""; - String url = logServerURL; - if (user != null && SliderUtils.isSet(url)) { - completedLogsUrl = url - + "/" + c.getNodeId() + "/" + c.getId() + "/ctx/" + user; - } - return completedLogsUrl; - } - - /** - * Return the percentage done that Slider is to have YARN display in its - * Web UI - * @return an number from 0 to 100 - */ - public synchronized float getApplicationProgressPercentage() { - float percentage; - long desired = 0; - float actual = 0; - for (RoleStatus role : getRoleStatusMap().values()) { - desired += role.getDesired(); - actual += role.getRunning(); - } - if (desired == 0) { - percentage = 100; - } else { - percentage = actual / desired; - } - return percentage; - } - - - /** - * Update the cluster description with the current application state - */ - - public synchronized Application refreshClusterStatus() { - - //TODO replace ClusterDescription with Application + related statistics - //TODO build container stats - app.setState(ApplicationState.STARTED); - return app; - } - - /** - * get application liveness information - * @return a snapshot of the current liveness information - */ - public ApplicationLivenessInformation getApplicationLivenessInformation() { - ApplicationLivenessInformation li = new ApplicationLivenessInformation(); - RoleStatistics stats = getRoleStatistics(); - int outstanding = (int)(stats.desired - stats.actual); - li.requestsOutstanding = outstanding; - li.allRequestsSatisfied = outstanding <= 0; - return li; - } - - - /** - * Get the aggregate statistics across all roles - * @return role statistics - */ - public RoleStatistics getRoleStatistics() { - RoleStatistics stats = new RoleStatistics(); - for (RoleStatus role : getRoleStatusMap().values()) { - stats.add(role.getStatistics()); - } - return stats; - } - - /** - * Get a snapshot of component information. - *

- * This does not include any container list, which - * is more expensive to create. - * @return a map of current role status values. - */ - public Map getComponentInfoSnapshot() { - - Map statusMap = getRoleStatusMap(); - Map results = new HashMap<>( - statusMap.size()); - - for (RoleStatus status : statusMap.values()) { - String name = status.getName(); - ComponentInformation info = status.serialize(); - results.put(name, info); - } - return results; - } - - public synchronized AbstractRMOperation updateBlacklist() { - UpdateBlacklistOperation blacklistOperation = - roleHistory.updateBlacklist(getRoleStatusMap().values()); - if (blacklistOperation != null) { - log.info("Updating {}", blacklistOperation); - } - return blacklistOperation; - } - - /** - * Look at where the current node state is -and whether it should be changed - */ - public synchronized List reviewRequestAndReleaseNodes() - throws SliderInternalStateException, TriggerClusterTeardownException { - log.info("in reviewRequestAndReleaseNodes()"); - List allOperations = new ArrayList<>(); - AbstractRMOperation blacklistOperation = updateBlacklist(); - if (blacklistOperation != null) { - allOperations.add(blacklistOperation); - } - for (RoleStatus roleStatus : getRoleStatusMap().values()) { - if (!roleStatus.isExcludeFromFlexing() && - areDependenciesReady(roleStatus)) { - List operations = reviewOneRole(roleStatus); - allOperations.addAll(operations); - } - } - return allOperations; - } - - @VisibleForTesting - public boolean areDependenciesReady(RoleStatus roleStatus) { - List dependencies = roleStatus.getProviderRole().component - .getDependencies(); - if (SliderUtils.isEmpty(dependencies)) { - return true; - } - for (String dependency : dependencies) { - ProviderRole providerRole = roles.get(dependency); - if (providerRole == null) { - log.error("Couldn't find dependency {} for {} (should never happen)", - dependency, roleStatus.getName()); - continue; - } - RoleStatus other = getRoleStatusMap().get(providerRole.id); - if (other.getRunning() < other.getDesired()) { - log.info("Dependency {} not satisfied for {}, only {} of {} instances" + - " running", dependency, roleStatus.getName(), other.getRunning(), - other.getDesired()); - return false; - } - if (providerRole.probe == null) { - continue; - } - List dependencyInstances = enumLiveNodesInRole( - providerRole.name); - if (dependencyInstances.size() < other.getDesired()) { - log.info("Dependency {} not satisfied for {}, only {} of {} instances" + - " live", dependency, roleStatus.getName(), - dependencyInstances.size(), other.getDesired()); - return false; - } - for (RoleInstance instance : dependencyInstances) { - if (instance.state != STATE_READY) { - return false; - } - } - } - return true; - } - - /** - * Check the "recent" failure threshold for a role - * @param role role to examine - * @throws TriggerClusterTeardownException if the role - * has failed too many times - */ - private void checkFailureThreshold(RoleStatus role) - throws TriggerClusterTeardownException { - long failures = role.getFailedRecently(); - int threshold = getFailureThresholdForRole(role); - if (log.isDebugEnabled() && failures > 0) { - log.debug("Failure count of component: {}: {}, threshold={}", - role.getName(), failures, threshold); - } - - if (threshold > 0 && failures > threshold) { - throw new TriggerClusterTeardownException( - SliderExitCodes.EXIT_DEPLOYMENT_FAILED, FinalApplicationStatus.FAILED, - ErrorStrings.E_UNSTABLE_CLUSTER - + " - failed with component %s failed 'recently' %d times;" - + " threshold is %d - last failure: %s", role.getName(), - role.getFailedRecently(), threshold, role.getFailureMessage()); - } - } - - /** - * Get the failure threshold for a specific role, falling back to - * the global one if not - * @param roleStatus role - * @return the threshold for failures - */ - private int getFailureThresholdForRole(RoleStatus roleStatus) { - return (int) roleStatus.getProviderRole().component.getConfiguration() - .getPropertyLong(CONTAINER_FAILURE_THRESHOLD, - failureThreshold); - } - - - /** - * Reset the "recent" failure counts of all roles - */ - public void resetFailureCounts() { - for (RoleStatus roleStatus : getRoleStatusMap().values()) { - long failed = roleStatus.resetFailedRecently(); - log.info("Resetting failure count of {}; was {}", roleStatus.getName(), - failed); - - } - roleHistory.resetFailedRecently(); - } - - /** - * Escalate operation as triggered by external timer. - * @return a (usually empty) list of cancel/request operations. - */ - public List escalateOutstandingRequests() { - return roleHistory.escalateOutstandingRequests(); - } - - /** - * Cancel any outstanding AA Requests, building up the list of ops to - * cancel, removing them from RoleHistory structures and the RoleStatus - * entries. - * @return a (usually empty) list of cancel/request operations. - */ - public synchronized List cancelOutstandingAARequests() { - // get the list of cancel operations - List operations = roleHistory.cancelOutstandingAARequests(); - for (RoleStatus roleStatus : roleStatusMap.values()) { - if (roleStatus.isAARequestOutstanding()) { - log.info("Cancelling outstanding AA request for {}", roleStatus); - roleStatus.cancelOutstandingAARequest(); - } - } - return operations; - } - - public synchronized boolean monitorComponentInstances() { - boolean hasChanged = false; - for (RoleInstance instance : getLiveContainers().values()) { - if (instance.providerRole.probe == null) { - continue; - } - boolean ready = instance.providerRole.probe.ping(null).isSuccess(); - if (ready) { - if (instance.state != STATE_READY) { - instance.state = STATE_READY; - hasChanged = true; - log.info("State of {} changed to ready", instance.role); - } - } else { - if (instance.state == STATE_READY) { - instance.state = STATE_NOT_READY; - hasChanged = true; - log.info("State of {} changed from ready to not ready", instance - .role); - } - } - } - return hasChanged; - } - - /** - * Look at the allocation status of one role, and trigger add/release - * actions if the number of desired role instances doesn't equal - * (actual + pending). - *

- * MUST be executed from within a synchronized method - *

- * @param role role - * @return a list of operations - * @throws SliderInternalStateException if the operation reveals that - * the internal state of the application is inconsistent. - */ - @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") - private List reviewOneRole(RoleStatus role) - throws SliderInternalStateException, TriggerClusterTeardownException { - List operations = new ArrayList<>(); - long delta; - long expected; - String name = role.getName(); - synchronized (role) { - delta = role.getDelta(); - expected = role.getDesired(); - } - - log.info("Reviewing " + role.getName() + ": " + role.getComponentMetrics()); - checkFailureThreshold(role); - - if (expected < 0 ) { - // negative value: fail - throw new TriggerClusterTeardownException( - SliderExitCodes.EXIT_DEPLOYMENT_FAILED, - FinalApplicationStatus.FAILED, - "Negative component count of %d desired for component %s", - expected, role); - } - - if (delta > 0) { - // more workers needed than we have -ask for more - log.info("{}: Asking for {} more nodes(s) for a total of {} ", name, delta, expected); - - if (role.isAntiAffinePlacement()) { - long pending = delta; - if (roleHistory.canPlaceAANodes()) { - // build one only if there is none outstanding, the role history knows - // enough about the cluster to ask, and there is somewhere to place - // the node - if (!role.isAARequestOutstanding()) { - // no outstanding AA; try to place things - AMRMClient.ContainerRequest request = createAAContainerRequest(role); - if (request != null) { - pending--; - log.info("Starting an anti-affine request sequence for {} nodes; pending={}", - delta, pending); - addContainerRequest(operations, request, role); - } else { - log.info("No location for anti-affine request"); - } - } - } else { - log.warn("Awaiting node map before generating anti-affinity requests"); - } - log.info("Setting pending to {}", pending); - //TODO - role.setAAPending(pending); - } else { - - for (int i = 0; i < delta; i++) { - //get the role history to select a suitable node, if available - addContainerRequest(operations, createContainerRequest(role), role); - } - } - } else if (delta < 0) { - log.info("{}: Asking for {} fewer node(s) for a total of {}", name, - -delta, - expected); - // reduce the number expected (i.e. subtract the delta) - long excess = -delta; - - // how many requests are outstanding? for AA roles, this includes pending - long outstandingRequests = role.getRequested() + role.getAAPending(); - if (outstandingRequests > 0) { - // outstanding requests. - int toCancel = (int)Math.min(outstandingRequests, excess); - - int pendingCancelled = 0; - if (role.getAAPending() > 0) { - pendingCancelled = decAAPendingToFloor(role, toCancel); - } - int remainingToCancel = toCancel - pendingCancelled; - - // Delegate to Role History - List cancellations = roleHistory - .cancelRequestsForRole(role, remainingToCancel); - log.info("Found {} outstanding requests to cancel", cancellations.size()); - operations.addAll(cancellations); - if (remainingToCancel != cancellations.size()) { - log.error("Tracking of outstanding requests is not in sync with the summary statistics:" + - " expected to be able to cancel {} requests, but got {}", - remainingToCancel, cancellations.size()); - } - - int requestCancelled = decRequestedContainersToFloor(role, - remainingToCancel); - excess -= pendingCancelled; - excess -= requestCancelled; - assert excess >= 0 : "Attempted to cancel too many requests"; - log.info("Submitted {} cancellations, leaving {} to release", - pendingCancelled + requestCancelled, excess); - if (excess == 0) { - log.info("After cancelling requests, application is now at desired size"); - } - } - - // after the cancellation there may be no excess - if (excess > 0) { - - // there's an excess, so more to cancel - // get the nodes to release - int roleId = role.getKey(); - - // enum all active nodes that aren't being released - List containersToRelease = enumNodesWithRoleId(roleId, true); - if (containersToRelease.isEmpty()) { - log.info("No containers for component {}", roleId); - } - - // filter out all release-in-progress nodes - ListIterator li = containersToRelease.listIterator(); - while (li.hasNext()) { - RoleInstance next = li.next(); - if (next.released) { - li.remove(); - } - } - - // warn if the desired state can't be reached - int numberAvailableForRelease = containersToRelease.size(); - if (numberAvailableForRelease < excess) { - log.warn("Not enough containers to release, have {} and need {} more", - numberAvailableForRelease, - excess - numberAvailableForRelease); - } - - // ask the release selector to sort the targets - containersToRelease = containerReleaseSelector.sortCandidates( - roleId, - containersToRelease); - - // crop to the excess - List finalCandidates = (excess < numberAvailableForRelease) - ? containersToRelease.subList(0, (int)excess) - : containersToRelease; - - // then build up a release operation, logging each container as released - for (RoleInstance possible : finalCandidates) { - log.info("Targeting for release: {}", possible); - containerReleaseSubmitted(possible.container); - role.getProviderRole().failedInstances.offer(possible); - operations.add(new ContainerReleaseOperation(possible.getContainerId())); - } - } - - } else { - // actual + requested == desired - // there's a special case here: clear all pending AA requests - if (role.getAAPending() > 0) { - log.debug("Clearing outstanding pending AA requests"); - role.setAAPending(0); - } - } - - // there's now a list of operations to execute - log.debug("operations scheduled: {}; updated role: {}", operations.size(), role); - return operations; - } - - /** - * Add a container request if the request is non-null - * @param operations operations to add the entry to - * @param containerAsk what to ask for - * @return true if a request was added - */ - private boolean addContainerRequest(List operations, - AMRMClient.ContainerRequest containerAsk, RoleStatus role) { - if (containerAsk != null) { - log.info("Container ask is {} and label = {}", containerAsk, - containerAsk.getNodeLabelExpression()); - int askMemory = containerAsk.getCapability().getMemory(); - if (askMemory > this.containerMaxMemory) { - log.warn("Memory requested: {} > max of {}", askMemory, containerMaxMemory); - } - operations.add(new ContainerRequestOperation(containerAsk)); - incRequestedContainers(role); - return true; - } else { - return false; - } - } - - /** - * Releases a container based on container id - * @param containerId - * @return - * @throws SliderInternalStateException - */ - public List releaseContainer(ContainerId containerId) - throws SliderInternalStateException { - List operations = new ArrayList(); - List activeRoleInstances = cloneOwnedContainerList(); - for (RoleInstance role : activeRoleInstances) { - if (role.container.getId().equals(containerId)) { - containerReleaseSubmitted(role.container); - operations.add(new ContainerReleaseOperation(role.getContainerId())); - } - } - - return operations; - } - - /** - * Release all containers. - * @return a list of operations to execute - */ - public synchronized List releaseAllContainers() { - - Collection targets = cloneOwnedContainerList(); - log.info("Releasing {} containers", targets.size()); - List operations = - new ArrayList<>(targets.size()); - for (RoleInstance instance : targets) { - if (instance.roleId == SliderKeys.ROLE_AM_PRIORITY_INDEX) { - // don't worry about the AM - continue; - } - Container possible = instance.container; - ContainerId id = possible.getId(); - if (!instance.released) { - String url = getLogsURLForContainer(possible); - log.info("Releasing container. Log: " + url); - try { - containerReleaseSubmitted(possible); - // update during finish call - if (serviceTimelinePublisher != null) { - serviceTimelinePublisher.componentInstanceFinished(instance); - } - } catch (SliderInternalStateException e) { - log.warn("when releasing container {} :", possible, e); - } - operations.add(new ContainerReleaseOperation(id)); - } - } - return operations; - } - - /** - * Event handler for allocated containers: builds up the lists - * of assignment actions (what to run where), and possibly - * a list of operations to perform - * @param allocatedContainers the containers allocated - * @param assignments the assignments of roles to containers - * @param operations any allocation or release operations - */ - public synchronized void onContainersAllocated( - List allocatedContainers, - List assignments, - List operations) { - assignments.clear(); - operations.clear(); - List ordered = roleHistory.prepareAllocationList(allocatedContainers); - log.info("onContainersAllocated(): Total containers allocated = {}", ordered.size()); - for (Container container : ordered) { - final NodeId nodeId = container.getNodeId(); - String containerHostInfo = nodeId.getHost() + ":" + nodeId.getPort(); - //get the role - final ContainerId cid = container.getId(); - final RoleStatus role = lookupRoleStatus(container); - - //inc allocated count -this may need to be dropped in a moment, - // but us needed to update the logic below - MutableGaugeInt containersRunning = role.getComponentMetrics().containersRunning; - incRunningContainers(role); - final long allocated = containersRunning.value(); - final long desired = role.getDesired(); - - final String roleName = role.getName(); - final ContainerAllocationResults allocation = - roleHistory.onContainerAllocated(container, desired, allocated); - final ContainerAllocationOutcome outcome = allocation.outcome; - - // add all requests to the operations list - operations.addAll(allocation.operations); - - //look for condition where we get more back than we asked - if (allocated > desired) { - log.info("Discarding surplus {} container {} on {}", roleName, cid, containerHostInfo); - operations.add(new ContainerReleaseOperation(cid)); - //register as a surplus node - surplusContainers.add(cid); - role.getComponentMetrics().surplusContainers.incr(); - containersRunning.decr(); - } else { - decRequestedContainers(role); - log.info("Assigning role {} to container" + " {}," + " on {}:{},", - roleName, cid, nodeId.getHost(), nodeId.getPort()); - - assignments.add(new ContainerAssignment(container, role, outcome)); - //add to the history - roleHistory.onContainerAssigned(container); - // now for AA requests, add some more - if (role.isAntiAffinePlacement()) { - role.completeOutstandingAARequest(); - // check invariants. The new node must become unavailable. - NodeInstance node = roleHistory.getOrCreateNodeInstance(container); - if (node.canHost(role.getKey(), role.getLabelExpression())) { - log.error("Assigned node still declares as available {}", node.toFullString() ); - } - if (role.getAAPending() > 0) { - // still an outstanding AA request: need to issue a new one. - log.info("Asking for next container for AA role {}", roleName); - if (!addContainerRequest(operations, createAAContainerRequest(role), - role)) { - log.info("No capacity in cluster for new requests"); - } else { - role.decAAPending(); - } - log.debug("Current AA role status {}", role); - } else { - log.info("AA request sequence completed for role {}", role); - } - } - - } - } - } - - /** - * Event handler for the list of active containers on restart. - * Sets the info key {@link StatusKeys#INFO_CONTAINERS_AM_RESTART} - * to the size of the list passed down (and does not set it if none were) - * @param liveContainers the containers allocated - * @return true if a rebuild took place (even if size 0) - * @throws RuntimeException on problems - */ - private boolean rebuildModelFromRestart(List liveContainers) - throws BadClusterStateException { - if (liveContainers == null) { - return false; - } - for (Container container : liveContainers) { - addRestartedContainer(container); - } - app.setNumberOfRunningContainers((long)liveContainers.size()); - return true; - } - - /** - * Add a restarted container by walking it through the create/submit/start - * lifecycle, so building up the internal structures - * @param container container that was running before the AM restarted - * @throws RuntimeException on problems - */ - private void addRestartedContainer(Container container) - throws BadClusterStateException { - String containerHostInfo = container.getNodeId().getHost() - + ":" + - container.getNodeId().getPort(); - // get the container ID - ContainerId cid = container.getId(); - - // get the role - int roleId = ContainerPriority.extractRole(container); - RoleStatus role = lookupRoleStatus(roleId); - // increment its count - incRunningContainers(role); - String roleName = role.getName(); - - log.info("Rebuilding container {} in role {} on {},", - cid, - roleName, - containerHostInfo); - - //update app state internal structures and maps - - //TODO recover the component instance name from zk registry ? - RoleInstance instance = new RoleInstance(container); - instance.command = roleName; - instance.role = roleName; - instance.roleId = roleId; - instance.environment = new String[0]; - instance.container = container; - instance.createTime = now(); - instance.state = STATE_LIVE; - instance.appVersion = SliderKeys.APP_VERSION_UNKNOWN; - putOwnedContainer(cid, instance); - //role history gets told - roleHistory.onContainerAssigned(container); - // pretend the container has just had its start actions submitted - containerStartSubmitted(container, instance); - // now pretend it has just started - innerOnNodeManagerContainerStarted(cid); - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder("AppState{"); - sb.append("applicationLive=").append(applicationLive); - sb.append(", live nodes=").append(liveNodes.size()); - sb.append('}'); - return sb.toString(); - } - - /** - * Build map of role ID-> name - * @return - */ - public Map buildNamingMap() { - Map statusMap = getRoleStatusMap(); - Map naming = new HashMap<>(statusMap.size()); - for (Map.Entry entry : statusMap.entrySet()) { - naming.put(entry.getKey(), entry.getValue().getName()); - } - return naming; - } - - public void setServiceTimelinePublisher(ServiceTimelinePublisher serviceTimelinePublisher) { - this.serviceTimelinePublisher = serviceTimelinePublisher; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java deleted file mode 100644 index 439a256..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.slider.api.resource.Application; -import org.apache.slider.providers.ProviderRole; - -import java.util.ArrayList; -import java.util.List; - -/** - * Binding information for application states; designed to be extensible - * so that tests don't have to be massivley reworked when new arguments - * are added. - */ -public class AppStateBindingInfo { - public Configuration serviceConfig = new Configuration(); - public Application application = null; - public List roles = new ArrayList<>(); - public FileSystem fs; - public Path historyPath; - public List liveContainers = new ArrayList<>(0); - public ContainerReleaseSelector releaseSelector = new SimpleReleaseSelector(); - public String serviceHdfsDir = ""; - /** node reports off the RM. */ - public List nodeReports = new ArrayList<>(0); - - public void validate() throws IllegalArgumentException { - Preconditions.checkArgument(serviceConfig != null, "null appmasterConfig"); - Preconditions.checkArgument(releaseSelector != null, "null releaseSelector"); - Preconditions.checkArgument(roles != null, "null providerRoles"); - Preconditions.checkArgument(fs != null, "null fs"); - Preconditions.checkArgument(historyPath != null, "null historyDir"); - Preconditions.checkArgument(nodeReports != null, "null nodeReports"); - Preconditions.checkArgument(application != null, "null application"); - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationOutcome.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationOutcome.java deleted file mode 100644 index 5b3a93c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationOutcome.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -/** - * Outcome of the assignment - */ -public enum ContainerAllocationOutcome { - /** - * There wasn't a request for this - */ - Unallocated, - - /** - * Open placement - */ - Open, - - /** - * Allocated explicitly where requested - */ - Placed, - - /** - * This was an escalated placement - */ - Escalated -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationResults.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationResults.java deleted file mode 100644 index e80639e..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationResults.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; - -import java.util.ArrayList; -import java.util.List; - -/** - * This is just a tuple of the outcome of a container allocation - */ -public class ContainerAllocationResults { - - /** - * What was the outcome of this allocation: placed, escalated, ... - */ - public ContainerAllocationOutcome outcome; - - /** - * The outstanding request which originated this. - * This will be null if the outcome is {@link ContainerAllocationOutcome#Unallocated} - * as it wasn't expected. - */ - public OutstandingRequest origin; - - /** - * A possibly empty list of requests to add to the follow-up actions - */ - public List operations = new ArrayList<>(0); - - public ContainerAllocationResults() { - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAssignment.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAssignment.java deleted file mode 100644 index 3e8a3c3..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAssignment.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import org.apache.hadoop.yarn.api.records.Container; - -/** - * Static assignment structure - */ -public class ContainerAssignment { - - /** - * Container that has been allocated - */ - public final Container container; - - /** - * Role to assign to it - */ - public final RoleStatus role; - - /** - * Placement outcome: was this from history or not - */ - public final ContainerAllocationOutcome placement; - - public ContainerAssignment(Container container, - RoleStatus role, - ContainerAllocationOutcome placement) { - this.container = container; - this.role = role; - this.placement = placement; - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder("ContainerAssignment{"); - sb.append("container=").append(container); - sb.append(", role=").append(role); - sb.append(", placement=").append(placement); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerOutcome.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerOutcome.java deleted file mode 100644 index 6df4bf4..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerOutcome.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import org.apache.hadoop.yarn.api.records.ContainerExitStatus; - -/** - * Container outcomes we care about; slightly simplified from - * {@link ContainerExitStatus} -and hopefully able to handle - * any new exit codes. - */ -public enum ContainerOutcome { - Completed, - Failed, - Failed_limits_exceeded, - Disk_failure, - Preempted; - - /** - * Build a container outcome from an exit status. - * The values in {@link ContainerExitStatus} are used - * here. - * @param exitStatus exit status - * @return an enumeration of the outcome. - */ - public static ContainerOutcome fromExitStatus(int exitStatus) { - switch (exitStatus) { - case ContainerExitStatus.ABORTED: - case ContainerExitStatus.KILLED_BY_APPMASTER: - case ContainerExitStatus.KILLED_BY_RESOURCEMANAGER: - case ContainerExitStatus.KILLED_AFTER_APP_COMPLETION: - // could either be a release or node failure. Treat as completion - return Completed; - case ContainerExitStatus.DISKS_FAILED: - return Disk_failure; - case ContainerExitStatus.PREEMPTED: - return Preempted; - case ContainerExitStatus.KILLED_EXCEEDED_PMEM: - case ContainerExitStatus.KILLED_EXCEEDED_VMEM: - return Failed_limits_exceeded; - default: - return exitStatus == 0 ? Completed : Failed; - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerPriority.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerPriority.java deleted file mode 100644 index df222fa..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerPriority.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.util.Records; - -import java.util.Locale; - -/** - * Class containing the logic to build/split container priorities into the - * different fields used by Slider - * - * The original design here had a requestID merged with the role, to - * track outstanding requests. However, this isn't possible, so - * the request ID has been dropped. A "location specified" flag was - * added to indicate whether or not the request was for a specific location - * -though this is currently unused. - * - * The methods are effectively surplus -but retained to preserve the - * option of changing behavior in future - */ -public final class ContainerPriority { - - // bit that represents whether location is specified - static final int NOLOCATION = 1 << 30; - - public static int buildPriority(int role, - boolean locationSpecified) { - int location = locationSpecified ? 0 : NOLOCATION; - return role | location; - } - - - public static Priority createPriority(int role, - boolean locationSpecified) { - Priority pri = Records.newRecord(Priority.class); - pri.setPriority(ContainerPriority.buildPriority(role, - locationSpecified)); - return pri; - } - - public static int extractRole(int priority) { - return priority >= NOLOCATION ? priority ^ NOLOCATION : priority; - } - - /** - * Does the priority have location - * @param priority priority index - * @return true if the priority has the location marker - */ - public static boolean hasLocation(int priority) { - return (priority ^ NOLOCATION ) == 0; - } - - /** - * Map from a container to a role key by way of its priority - * @param container container - * @return role key - */ - public static int extractRole(Container container) { - Priority priority = container.getPriority(); - return extractRole(priority); - } - - /** - * Priority record to role mapper - * @param priorityRecord priority record - * @return the role # - */ - public static int extractRole(Priority priorityRecord) { - Preconditions.checkNotNull(priorityRecord); - return extractRole(priorityRecord.getPriority()); - } - - /** - * Convert a priority record to a string, extracting role and locality - * @param priorityRecord priority record. May be null - * @return a string value - */ - public static String toString(Priority priorityRecord) { - if (priorityRecord==null) { - return "(null)"; - } else { - return String.format(Locale.ENGLISH, - "role %d (locality=%b)", - extractRole(priorityRecord), - hasLocation(priorityRecord.getPriority())); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerReleaseSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerReleaseSelector.java deleted file mode 100644 index fafbada..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerReleaseSelector.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import java.util.List; - -/** - * Interface implemented by anything that must choose containers to release - * - */ -public interface ContainerReleaseSelector { - - /** - * Given a list of candidate containers, return a sorted version of the priority - * in which they should be released. - * @param candidates candidate list ... everything considered suitable - * @return the list of candidates - */ - List sortCandidates(int roleId, - List candidates); -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/MostRecentContainerReleaseSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/MostRecentContainerReleaseSelector.java deleted file mode 100644 index 38c5b8e..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/MostRecentContainerReleaseSelector.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import org.apache.slider.common.tools.Comparators; - -import java.io.Serializable; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; - -/** - * Sort the candidate list by the most recent container first. - */ -public class MostRecentContainerReleaseSelector implements ContainerReleaseSelector { - - @Override - public List sortCandidates(int roleId, - List candidates) { - Collections.sort(candidates, new newerThan()); - return candidates; - } - - private static class newerThan implements Comparator, Serializable { - private final Comparator innerComparator = - new Comparators.ComparatorReverser<>(new Comparators.LongComparator()); - public int compare(RoleInstance o1, RoleInstance o2) { - return innerComparator.compare(o1.createTime, o2.createTime); - - } - - } - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeEntry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeEntry.java deleted file mode 100644 index d57b6d2..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeEntry.java +++ /dev/null @@ -1,325 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.slider.api.types.NodeEntryInformation; - -/** - * Information about the state of a role on a specific node instance. - * No fields are synchronized; sync on the instance to work with it - *

- * The two fields `releasing` and `requested` are used to track the ongoing - * state of YARN requests; they do not need to be persisted across stop/start - * cycles. They may be relevant across AM restart, but without other data - * structures in the AM, not enough to track what the AM was up to before - * it was restarted. The strategy will be to ignore unexpected allocation - * responses (which may come from pre-restart) requests, while treating - * unexpected container release responses as failures. - *

- * The `active` counter is only decremented after a container release response - * has been received. - *

- * - */ -public class NodeEntry implements Cloneable { - - public final int rolePriority; - - public NodeEntry(int rolePriority) { - this.rolePriority = rolePriority; - } - - /** - * instance explicitly requested on this node: it's OK if an allocation - * comes in that has not been (and when that happens, this count should - * not drop). - */ - private int requested; - - /** number of starting instances */ - private int starting; - - /** incrementing counter of instances that failed to start */ - private int startFailed; - - /** incrementing counter of instances that failed */ - private int failed; - - /** - * Counter of "failed recently" events. These are all failures - * which have happened since it was last reset. - */ - private int failedRecently; - - /** incrementing counter of instances that have been pre-empted. */ - private int preempted; - - /** - * Number of live nodes. - */ - private int live; - - /** number of containers being released off this node */ - private int releasing; - - /** timestamp of last use */ - private long lastUsed; - - /** - * Is the node available for assignments? That is, it is - * not running any instances of this type, nor are there - * any requests oustanding for it. - * @return true if a new request could be issued without taking - * the number of instances > 1. - */ - public synchronized boolean isAvailable() { - return live + requested + starting - releasing <= 0; - } - - /** - * Are the anti-affinity constraints held. That is, zero or one - * node running or starting - * @return true if the constraint holds. - */ - public synchronized boolean isAntiAffinityConstraintHeld() { - return (live - releasing + starting) <= 1; - } - - /** - * return no of active instances -those that could be released as they - * are live and not already being released - * @return a number, possibly 0 - */ - public synchronized int getActive() { - return (live - releasing); - } - - /** - * Return true if the node is not busy, and it - * has not been used since the absolute time - * @param absoluteTime time - * @return true if the node could be cleaned up - */ - public synchronized boolean notUsedSince(long absoluteTime) { - return isAvailable() && lastUsed < absoluteTime; - } - - public synchronized int getLive() { - return live; - } - - public int getStarting() { - return starting; - } - - /** - * Set the live value directly -used on AM restart - * @param v value - */ - public synchronized void setLive(int v) { - live = v; - } - - private synchronized void incLive() { - ++live; - } - - private synchronized void decLive() { - live = RoleHistoryUtils.decToFloor(live); - } - - public synchronized void onStarting() { - ++starting; - } - - private void decStarting() { - starting = RoleHistoryUtils.decToFloor(starting); - } - - public synchronized void onStartCompleted() { - decStarting(); - incLive(); - } - - /** - * start failed -decrement the starting flag. - * @return true if the node is now available - */ - public synchronized boolean onStartFailed() { - decStarting(); - ++startFailed; - return containerCompleted(false, ContainerOutcome.Failed); - } - - /** - * no of requests made of this role of this node. If it goes above - * 1 there's a problem - */ - public synchronized int getRequested() { - return requested; - } - - /** - * request a node: - */ - public synchronized void request() { - ++requested; - } - - /** - * A request made explicitly to this node has completed - */ - public synchronized void requestCompleted() { - requested = RoleHistoryUtils.decToFloor(requested); - } - - /** - * No of instances in release state - */ - public synchronized int getReleasing() { - return releasing; - } - - /** - * Release an instance -which is no longer marked as active - */ - public synchronized void release() { - releasing++; - } - - /** - * completion event, which can be a planned or unplanned - * planned: dec our release count - * unplanned: dec our live count - * @param wasReleased true if this was planned - * @param outcome - * @return true if this node is now available - */ - public synchronized boolean containerCompleted(boolean wasReleased, ContainerOutcome outcome) { - if (wasReleased) { - releasing = RoleHistoryUtils.decToFloor(releasing); - } else { - // for the node, we use the outcome of the faiure to decide - // whether this is potentially "node-related" - switch(outcome) { - // general "any reason" app failure - case Failed: - // specific node failure - case Disk_failure: - - ++failed; - ++failedRecently; - break; - - case Preempted: - preempted++; - break; - - // failures which are node-independent - case Failed_limits_exceeded: - case Completed: - default: - break; - } - } - decLive(); - return isAvailable(); - } - - /** - * Time last used. - */ - public synchronized long getLastUsed() { - return lastUsed; - } - - public synchronized void setLastUsed(long lastUsed) { - this.lastUsed = lastUsed; - } - - public synchronized int getStartFailed() { - return startFailed; - } - - public synchronized int getFailed() { - return failed; - } - - public synchronized int getFailedRecently() { - return failedRecently; - } - - @VisibleForTesting - public synchronized void setFailedRecently(int failedRecently) { - this.failedRecently = failedRecently; - } - - public synchronized int getPreempted() { - return preempted; - } - - - /** - * Reset the failed recently count. - */ - public synchronized void resetFailedRecently() { - failedRecently = 0; - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder("NodeEntry{"); - sb.append("priority=").append(rolePriority); - sb.append(", requested=").append(requested); - sb.append(", starting=").append(starting); - sb.append(", live=").append(live); - sb.append(", releasing=").append(releasing); - sb.append(", lastUsed=").append(lastUsed); - sb.append(", failedRecently=").append(failedRecently); - sb.append(", preempted=").append(preempted); - sb.append(", startFailed=").append(startFailed); - sb.append('}'); - return sb.toString(); - } - - /** - * Produced a serialized form which can be served up as JSON - * @return a summary of the current role status. - */ - public synchronized NodeEntryInformation serialize() { - NodeEntryInformation info = new NodeEntryInformation(); - info.priority = rolePriority; - info.requested = requested; - info.releasing = releasing; - info.starting = starting; - info.startFailed = startFailed; - info.failed = failed; - info.failedRecently = failedRecently; - info.preempted = preempted; - info.live = live; - info.lastUsed = lastUsed; - return info; - } - - @Override - public Object clone() throws CloneNotSupportedException { - return super.clone(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java deleted file mode 100644 index 120d402..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java +++ /dev/null @@ -1,419 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.NodeState; -import org.apache.slider.api.types.NodeInformation; -import org.apache.slider.common.tools.Comparators; -import org.apache.slider.common.tools.SliderUtils; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.ListIterator; -import java.util.Map; - -/** - * A node instance -stores information about a node in the cluster. - *

- * Operations on the array/set of roles are synchronized. - */ -public class NodeInstance { - - public final String hostname; - - private boolean blacklisted = false; - - /** - * last state of node. Starts off as {@link NodeState#RUNNING}, - * on the assumption that it is live. - */ - private NodeState nodeState = NodeState.RUNNING; - - /** - * Last node report. If null: none - */ - private NodeReport nodeReport = null; - - /** - * time of state update - */ - private long nodeStateUpdateTime = 0; - - /** - * Node labels. - * - * IMPORTANT: we assume that there is one label/node, which is the policy - * for Hadoop as of November 2015 - */ - private String nodeLabels = ""; - - /** - * An unordered list of node entries of specific roles. There's nothing - * indexed so as to support sparser datastructures. - */ - private final List nodeEntries; - - /** - * Create an instance and the (empty) array of nodes - * @param roles role count -the no. of roles - */ - public NodeInstance(String hostname, int roles) { - this.hostname = hostname; - nodeEntries = new ArrayList<>(roles); - } - - public synchronized void setBlacklisted(boolean blacklisted) { - this.blacklisted = blacklisted; - } - - public boolean isBlacklisted() { - return blacklisted; - } - - /** - * Update the node status. - * The return code is true if the node state changed enough to - * trigger a re-evaluation of pending requests. That is, either a node - * became available when it was previously not, or the label changed - * on an available node. - * - * Transitions of a node from live to dead aren't treated as significant, - * nor label changes on a dead node. - * - * @param report latest node report - * @return true if the node state changed enough for a request evaluation. - */ - public synchronized boolean updateNode(NodeReport report) { - nodeStateUpdateTime = report.getLastHealthReportTime(); - nodeReport = report; - NodeState oldState = nodeState; - boolean oldStateUnusable = oldState.isUnusable(); - nodeState = report.getNodeState(); - boolean newUsable = !nodeState.isUnusable(); - boolean nodeNowAvailable = oldStateUnusable && newUsable; - String labels = this.nodeLabels; - nodeLabels = SliderUtils.extractNodeLabel(report); - return nodeNowAvailable - || newUsable && !this.nodeLabels.equals(labels); - } - - public String getNodeLabels() { - return nodeLabels; - } - - /** - * Get the entry for a role -if present - * @param role role index - * @return the entry - * null if the role is out of range - */ - public synchronized NodeEntry get(int role) { - for (NodeEntry nodeEntry : nodeEntries) { - if (nodeEntry.rolePriority == role) { - return nodeEntry; - } - } - return null; - } - - /** - * Get the entry for a role -if present - * @param role role index - * @return the entry - * @throws ArrayIndexOutOfBoundsException if the role is out of range - */ - public synchronized NodeEntry getOrCreate(int role) { - NodeEntry entry = get(role); - if (entry == null) { - entry = new NodeEntry(role); - nodeEntries.add(entry); - } - return entry; - } - - /** - * Get the node entry matching a container on this node - * @param container container - * @return matching node instance for the role - */ - public NodeEntry getOrCreate(Container container) { - return getOrCreate(ContainerPriority.extractRole(container)); - } - - /** - * Count the number of active role instances on this node - * @param role role index - * @return 0 if there are none, otherwise the #of nodes that are running and - * not being released already. - */ - public int getActiveRoleInstances(int role) { - NodeEntry nodeEntry = get(role); - return (nodeEntry != null ) ? nodeEntry.getActive() : 0; - } - - /** - * Count the number of live role instances on this node - * @param role role index - * @return 0 if there are none, otherwise the #of nodes that are running - */ - public int getLiveRoleInstances(int role) { - NodeEntry nodeEntry = get(role); - return (nodeEntry != null ) ? nodeEntry.getLive() : 0; - } - - /** - * Is the node considered online - * @return the node - */ - public boolean isOnline() { - return !nodeState.isUnusable(); - } - - /** - * Query for a node being considered unreliable - * @param role role key - * @param threshold threshold above which a node is considered unreliable - * @return true if the node is considered unreliable - */ - public boolean isConsideredUnreliable(int role, int threshold) { - NodeEntry entry = get(role); - return entry != null && entry.getFailedRecently() > threshold; - } - - /** - * Get the entry for a role -and remove it if present - * @param role the role index - * @return the entry that WAS there - */ - public synchronized NodeEntry remove(int role) { - NodeEntry nodeEntry = get(role); - if (nodeEntry != null) { - nodeEntries.remove(nodeEntry); - } - return nodeEntry; - } - - public synchronized void set(int role, NodeEntry nodeEntry) { - remove(role); - nodeEntries.add(nodeEntry); - } - - /** - * run through each entry; gc'ing & removing old ones that don't have - * a recent failure count (we care about those) - * @param absoluteTime age in millis - * @return true if there are still entries left - */ - public synchronized boolean purgeUnusedEntries(long absoluteTime) { - boolean active = false; - ListIterator entries = nodeEntries.listIterator(); - while (entries.hasNext()) { - NodeEntry entry = entries.next(); - if (entry.notUsedSince(absoluteTime) && entry.getFailedRecently() == 0) { - entries.remove(); - } else { - active = true; - } - } - return active; - } - - - /** - * run through each entry resetting the failure count - */ - public synchronized void resetFailedRecently() { - for (NodeEntry entry : nodeEntries) { - entry.resetFailedRecently(); - } - } - - @Override - public String toString() { - return hostname; - } - - /** - * Full dump of entry including children - * @return a multi-line description fo the node - */ - public String toFullString() { - final StringBuilder sb = - new StringBuilder(toString()); - sb.append("{ "); - for (NodeEntry entry : nodeEntries) { - sb.append(String.format("%n [%02d] ", entry.rolePriority)); - sb.append(entry.toString()); - } - sb.append("} "); - return sb.toString(); - } - - /** - * Equality test is purely on the hostname of the node address - * @param o other - * @return true if the hostnames are equal - */ - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - NodeInstance that = (NodeInstance) o; - return hostname.equals(that.hostname); - } - - @Override - public int hashCode() { - return hostname.hashCode(); - } - - - /** - * Predicate to query if the number of recent failures of a role - * on this node exceeds that role's failure threshold. - * If there is no record of a deployment of that role on this - * node, the failure count is taken as "0". - * @param role role to look up - * @return true if the failure rate is above the threshold. - */ - public boolean exceedsFailureThreshold(RoleStatus role) { - NodeEntry entry = get(role.getKey()); - int numFailuresOnLastHost = entry != null ? entry.getFailedRecently() : 0; - int failureThreshold = role.getNodeFailureThreshold(); - return failureThreshold < 0 || numFailuresOnLastHost > failureThreshold; - } - - /** - * Produced a serialized form which can be served up as JSON - * @param naming map of priority -> value for naming entries - * @return a summary of the current role status. - */ - public synchronized NodeInformation serialize(Map naming) { - NodeInformation info = new NodeInformation(); - info.hostname = hostname; - // null-handling state constructor - info.state = "" + nodeState; - info.lastUpdated = nodeStateUpdateTime; - info.labels = nodeLabels; - if (nodeReport != null) { - info.httpAddress = nodeReport.getHttpAddress(); - info.rackName = nodeReport.getRackName(); - info.healthReport = nodeReport.getHealthReport(); - } - info.entries = new HashMap<>(nodeEntries.size()); - for (NodeEntry nodeEntry : nodeEntries) { - String name = naming.get(nodeEntry.rolePriority); - if (name == null) { - name = Integer.toString(nodeEntry.rolePriority); - } - info.entries.put(name, nodeEntry.serialize()); - } - return info; - } - - /** - * Is this node instance a suitable candidate for the specific role? - * @param role role ID - * @param label label which must match, or "" for no label checks - * @return true if the node has space for this role, is running and the labels - * match. - */ - public boolean canHost(int role, String label) { - return isOnline() - && (SliderUtils.isUnset(label) || label.equals(nodeLabels)) // label match - && getOrCreate(role).isAvailable(); // no live role - } - - /** - * A comparator for sorting entries where the node is preferred over another. - * - * The exact algorithm may change: current policy is "most recent first", so sorted - * on the lastUsed - * - * the comparision is a positive int if left is preferred to right; - * negative if right over left, 0 for equal - */ - public static class Preferred implements Comparator, Serializable { - - private static final Comparators.InvertedLongComparator comparator = - new Comparators.InvertedLongComparator(); - private final int role; - - public Preferred(int role) { - this.role = role; - } - - @Override - public int compare(NodeInstance o1, NodeInstance o2) { - NodeEntry left = o1.get(role); - NodeEntry right = o2.get(role); - long ageL = left != null ? left.getLastUsed() : -1; - long ageR = right != null ? right.getLastUsed() : -1; - return comparator.compare(ageL, ageR); - } - } - - /** - * A comparator for sorting entries where the role is newer than - * the other. - * This sort only compares the lastUsed field, not whether the - * node is in use or not - */ - public static class MoreActiveThan implements Comparator, - Serializable { - - private final int role; - - public MoreActiveThan(int role) { - this.role = role; - } - - @Override - public int compare(NodeInstance left, NodeInstance right) { - int activeLeft = left.getActiveRoleInstances(role); - int activeRight = right.getActiveRoleInstances(role); - return activeRight - activeLeft; - } - } - /** - * A comparator for sorting entries alphabetically - */ - public static class CompareNames implements Comparator, - Serializable { - - public CompareNames() { - } - - @Override - public int compare(NodeInstance left, NodeInstance right) { - return left.hostname.compareTo(right.hostname); - } - } - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeMap.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeMap.java deleted file mode 100644 index 3858b68..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeMap.java +++ /dev/null @@ -1,174 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * Node map map -and methods to work with it. - * Not Synchronized: caller is expected to lock access. - */ -public class NodeMap extends HashMap { - protected static final Logger log = - LoggerFactory.getLogger(NodeMap.class); - - /** - * number of roles - */ - private final int roleSize; - - /** - * Construct - * @param roleSize number of roles - */ - public NodeMap(int roleSize) { - this.roleSize = roleSize; - } - - /** - * Get the node instance for the specific node -creating it if needed - * @param hostname node - * @return the instance - */ - public NodeInstance getOrCreate(String hostname) { - NodeInstance node = get(hostname); - if (node == null) { - node = new NodeInstance(hostname, roleSize); - put(hostname, node); - } - return node; - } - - /** - * List the active nodes - * @param role role - * @return a possibly empty sorted list of all nodes that are active - * in that role - */ - public List listActiveNodes(int role) { - List nodes = new ArrayList<>(); - for (NodeInstance instance : values()) { - if (instance.getActiveRoleInstances(role) > 0) { - nodes.add(instance); - } - } - Collections.sort(nodes, new NodeInstance.MoreActiveThan(role)); - return nodes; - } - - /** - * reset the failed recently counters - */ - public void resetFailedRecently() { - for (Map.Entry entry : entrySet()) { - NodeInstance ni = entry.getValue(); - ni.resetFailedRecently(); - } - } - - /** - * Update the node state. Return true if the node state changed: either by - * being created, or by changing its internal state as defined - * by {@link NodeInstance#updateNode(NodeReport)}. - * - * @param hostname host name - * @param report latest node report - * @return true if the node state changed enough for a request evaluation. - */ - public boolean updateNode(String hostname, NodeReport report) { - boolean nodeExisted = get(hostname) != null; - boolean updated = getOrCreate(hostname).updateNode(report); - return updated || !nodeExisted; - } - - /** - * Clone point - * @return a shallow clone - */ - @Override - public Object clone() { - return super.clone(); - } - - /** - * Insert a list of nodes into the map; overwrite any with that name - * This is a bulk operation for testing. - * @param nodes collection of nodes. - */ - @VisibleForTesting - public void insert(Collection nodes) { - for (NodeInstance node : nodes) { - put(node.hostname, node); - } - } - - /** - * Test helper: build or update a cluster from a list of node reports - * @param reports the list of reports - * @return true if this has been considered to have changed the cluster - */ - @VisibleForTesting - public boolean buildOrUpdate(List reports) { - boolean updated = false; - for (NodeReport report : reports) { - updated |= getOrCreate(report.getNodeId().getHost()).updateNode(report); - } - return updated; - } - - /** - * Scan the current node map for all nodes capable of hosting an instance - * @param role role ID - * @param label label which must match, or "" for no label checks - * @return a possibly empty list of node instances matching the criteria. - */ - public List findAllNodesForRole(int role, String label) { - List nodes = new ArrayList<>(size()); - for (NodeInstance instance : values()) { - if (instance.canHost(role, label)) { - nodes.add(instance); - } - } - Collections.sort(nodes, new NodeInstance.CompareNames()); - return nodes; - } - - @Override - public synchronized String toString() { - final StringBuilder sb = new StringBuilder("NodeMap{"); - List keys = new ArrayList<>(keySet()); - Collections.sort(keys); - for (String key : keys) { - sb.append(key).append(": "); - sb.append(get(key).toFullString()).append("\n"); - } - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java deleted file mode 100644 index 4357ef8..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java +++ /dev/null @@ -1,428 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.hadoop.yarn.client.api.InvalidContainerRequestException; -import org.apache.hadoop.yarn.util.resource.Resources; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.server.appmaster.operations.CancelSingleRequest; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; - -/** - * Tracks an outstanding request. This is used to correlate an allocation response - * with the node and role used in the request. - *

- * The node identifier may be null -which indicates that a request was made without - * a specific target node - *

- * Equality and the hash code are based only on the role and hostname, - * which are fixed in the constructor. This means that a simple - * instance constructed with (role, hostname) can be used to look up - * a complete request instance in the {@link OutstandingRequestTracker} map - */ -public final class OutstandingRequest extends RoleHostnamePair { - protected static final Logger log = - LoggerFactory.getLogger(OutstandingRequest.class); - - /** - * Node the request is for -may be null - */ - public final NodeInstance node; - - /** - * A list of all possible nodes to list in an AA request. For a non-AA - * request where {@link #node} is set, element 0 of the list is the same - * value. - */ - public final List nodes = new ArrayList<>(1); - - /** - * Optional label. This is cached as the request option (explicit-location + label) is forbidden, - * yet the label needs to be retained for escalation. - */ - public String label; - - /** - * Requested time in millis. - *

- * Only valid after {@link #buildContainerRequest(Resource, RoleStatus, long)} - */ - private AMRMClient.ContainerRequest issuedRequest; - - /** - * Requested time in millis. - *

- * Only valid after {@link #buildContainerRequest(Resource, RoleStatus, long)} - */ - private long requestedTimeMillis; - - /** - * Time in millis after which escalation should be triggered.. - *

- * Only valid after {@link #buildContainerRequest(Resource, RoleStatus, long)} - */ - private long escalationTimeoutMillis; - - /** - * Has the placement request been escalated? - */ - private boolean escalated; - - /** - * Flag to indicate that escalation is allowed - */ - private boolean mayEscalate; - - /** - * Priority of request; only valid after the request is built up - */ - private int priority = -1; - - /** - * Is this an Anti-affine request which should be cancelled on - * a cluster resize? - */ - private boolean antiAffine = false; - - /** - * Create a request - * @param roleId role - * @param node node -can be null - */ - public OutstandingRequest(int roleId, - NodeInstance node) { - super(roleId, node != null ? node.hostname : null); - this.node = node; - nodes.add(node); - } - - /** - * Create an outstanding request with the given role and hostname - * Important: this is useful only for map lookups -the other constructor - * with the NodeInstance parameter is needed to generate node-specific - * container requests - * @param roleId role - * @param hostname hostname - */ - public OutstandingRequest(int roleId, String hostname) { - super(roleId, hostname); - this.node = null; - } - - /** - * Create an Anti-affine reques, including all listed nodes (there must be one) - * as targets. - * @param roleId role - * @param nodes list of nodes - */ - public OutstandingRequest(int roleId, List nodes) { - super(roleId, nodes.get(0).hostname); - this.node = null; - this.antiAffine = true; - this.nodes.addAll(nodes); - } - - /** - * Is the request located in the cluster, that is: does it have a node. - * @return true if a node instance was supplied in the constructor - */ - public boolean isLocated() { - return node != null; - } - - public long getRequestedTimeMillis() { - return requestedTimeMillis; - } - - public long getEscalationTimeoutMillis() { - return escalationTimeoutMillis; - } - - public synchronized boolean isEscalated() { - return escalated; - } - - public boolean mayEscalate() { - return mayEscalate; - } - - public AMRMClient.ContainerRequest getIssuedRequest() { - return issuedRequest; - } - - public int getPriority() { - return priority; - } - - public boolean isAntiAffine() { - return antiAffine; - } - - public void setAntiAffine(boolean antiAffine) { - this.antiAffine = antiAffine; - } - - /** - * Build a container request. - *

- * The value of {@link #node} is used to direct a lot of policy. If null, - * placement is relaxed. - * If not null, the choice of whether to use the suggested node - * is based on the placement policy and failure history. - *

- * If the request has an address, it is set in the container request - * (with a flag to enable relaxed priorities). - *

- * This operation sets the requested time flag, used for tracking timeouts - * on outstanding requests - * @param resource resource - * @param role role - * @param time time in millis to record as request time - * @return the request to raise - */ - public synchronized AMRMClient.ContainerRequest buildContainerRequest( - Resource resource, RoleStatus role, long time) { - Preconditions.checkArgument(resource != null, "null `resource` arg"); - Preconditions.checkArgument(role != null, "null `role` arg"); - - // cache label for escalation - label = role.getLabelExpression(); - requestedTimeMillis = time; - escalationTimeoutMillis = time + role.getPlacementTimeoutSeconds() * 1000; - String[] hosts; - boolean relaxLocality; - boolean strictPlacement = role.isStrictPlacement(); - NodeInstance target = this.node; - String nodeLabels; - - if (isAntiAffine()) { - int size = nodes.size(); - log.info("Creating anti-affine request across {} nodes; first node = {}", - size, hostname); - hosts = new String[size]; - StringBuilder builder = new StringBuilder(size * 16); - int c = 0; - for (NodeInstance nodeInstance : nodes) { - hosts[c++] = nodeInstance.hostname; - builder.append(nodeInstance.hostname).append(" "); - } - log.debug("Full host list: [ {}]", builder); - escalated = false; - mayEscalate = false; - relaxLocality = false; - nodeLabels = null; - } else if (target != null) { - // placed request. Hostname is used in request - hosts = new String[1]; - hosts[0] = target.hostname; - // and locality flag is set to false; Slider will decide when - // to relax things - relaxLocality = false; - - log.info("Submitting request for container on {}", hosts[0]); - // enable escalation for all but strict placements. - escalated = false; - mayEscalate = !strictPlacement; - nodeLabels = null; - } else { - // no hosts - hosts = null; - // relax locality is mandatory on an unconstrained placement - relaxLocality = true; - // declare that the the placement is implicitly escalated. - escalated = true; - // and forbid it happening - mayEscalate = false; - nodeLabels = label; - } - Priority pri = ContainerPriority.createPriority(roleId, !relaxLocality); - priority = pri.getPriority(); - issuedRequest = new AMRMClient.ContainerRequest(resource, - hosts, - null, - pri, - relaxLocality, - nodeLabels); - validate(); - return issuedRequest; - } - - - /** - * Build an escalated container request, updating {@link #issuedRequest} with - * the new value. - * @return the new container request, which has the same resource and label requirements - * as the original one, and the same host, but: relaxed placement, and a changed priority - * so as to place it into the relaxed list. - */ - public synchronized AMRMClient.ContainerRequest escalate() { - Preconditions.checkNotNull(issuedRequest, "cannot escalate if request not issued " + this); - log.debug("Escalating {}", this.toString()); - escalated = true; - - // this is now the priority - // it is tagged as unlocated because it needs to go into a different - // set of outstanding requests from the strict placements - Priority pri = ContainerPriority.createPriority(roleId, false); - // update the field - priority = pri.getPriority(); - - String[] nodes; - List issuedRequestNodes = issuedRequest.getNodes(); - if (SliderUtils.isUnset(label) && issuedRequestNodes != null) { - nodes = issuedRequestNodes.toArray(new String[issuedRequestNodes.size()]); - } else { - nodes = null; - } - - issuedRequest = new AMRMClient.ContainerRequest(issuedRequest.getCapability(), - nodes, - null, - pri, - true, - label); - validate(); - return issuedRequest; - } - - /** - * Mark the request as completed (or canceled). - *

- * Current action: if a node is defined, its request count is decremented - */ - public void completed() { - if (node != null) { - node.getOrCreate(roleId).requestCompleted(); - } - } - - /** - * Query to see if the request is available and ready to be escalated - * @param time time to check against - * @return true if escalation should begin - */ - public synchronized boolean shouldEscalate(long time) { - return mayEscalate - && !escalated - && issuedRequest != null - && escalationTimeoutMillis < time; - } - - /** - * Query for the resource requirements matching; always false before a request is issued - * @param resource - * @return - */ - public synchronized boolean resourceRequirementsMatch(Resource resource) { - return issuedRequest != null && Resources.fitsIn(issuedRequest.getCapability(), resource); - } - - @Override - public String toString() { - boolean requestHasLocation = ContainerPriority.hasLocation(getPriority()); - final StringBuilder sb = new StringBuilder("OutstandingRequest{"); - sb.append("roleId=").append(roleId); - if (hostname != null) { - sb.append(", hostname='").append(hostname).append('\''); - } - sb.append(", node=").append(node); - sb.append(", hasLocation=").append(requestHasLocation); - sb.append(", label=").append(label); - sb.append(", requestedTimeMillis=").append(requestedTimeMillis); - sb.append(", mayEscalate=").append(mayEscalate); - sb.append(", escalated=").append(escalated); - sb.append(", escalationTimeoutMillis=").append(escalationTimeoutMillis); - sb.append(", issuedRequest=").append( - issuedRequest != null ? SliderUtils.requestToString(issuedRequest) : "(null)"); - sb.append('}'); - return sb.toString(); - } - - /** - * Create a cancel operation - * @return an operation that can be used to cancel the request - */ - public CancelSingleRequest createCancelOperation() { - Preconditions.checkState(issuedRequest != null, "No issued request to cancel"); - return new CancelSingleRequest(issuedRequest); - } - - /** - * Valid if a node label expression specified on container request is valid or - * not. Mimics the logic in AMRMClientImpl, so can be used for preflight checking - * and in mock tests - * - */ - public void validate() throws InvalidContainerRequestException { - Preconditions.checkNotNull(issuedRequest, "request has not yet been built up"); - AMRMClient.ContainerRequest containerRequest = issuedRequest; - String requestDetails = this.toString(); - validateContainerRequest(containerRequest, priority, requestDetails); - } - - /** - * Inner Validation logic for container request - * @param containerRequest request - * @param priority raw priority of role - * @param requestDetails details for error messages - */ - @VisibleForTesting - public static void validateContainerRequest(AMRMClient.ContainerRequest containerRequest, - int priority, String requestDetails) { - String exp = containerRequest.getNodeLabelExpression(); - boolean hasRacks = containerRequest.getRacks() != null && - (!containerRequest.getRacks().isEmpty()); - boolean hasNodes = containerRequest.getNodes() != null && - (!containerRequest.getNodes().isEmpty()); - - boolean hasLabel = SliderUtils.isSet(exp); - - // Don't support specifying >= 2 node labels in a node label expression now - if (hasLabel && (exp.contains("&&") || exp.contains("||"))) { - throw new InvalidContainerRequestException( - "Cannot specify more than two node labels" - + " in a single node label expression: " + requestDetails); - } - - // Don't allow specify node label against ANY request listing hosts or racks - if (hasLabel && ( hasRacks || hasNodes)) { - throw new InvalidContainerRequestException( - "Cannot specify node label with rack or node: " + requestDetails); - } - } - - /** - * Create a new role/hostname pair for indexing. - * @return a new index. - */ - public RoleHostnamePair getIndex() { - return new RoleHostnamePair(roleId, hostname); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java deleted file mode 100644 index dbdf8ca..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java +++ /dev/null @@ -1,482 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.CancelSingleRequest; -import org.apache.slider.server.appmaster.operations.ContainerRequestOperation; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.ListIterator; -import java.util.Map; -import java.util.Set; - -/** - * Tracks outstanding requests made with a specific placement option. - *

- *

    - *
  1. Used to decide when to return a node to 'can request containers here' list
  2. - *
  3. Used to identify requests where placement has timed out, and so issue relaxed requests
  4. - *
- *

- * If an allocation comes in that is not in the map: either the allocation - * was unplaced, or the placed allocation could not be met on the specified - * host, and the RM/scheduler fell back to another location. - */ - -public class OutstandingRequestTracker { - protected static final Logger log = - LoggerFactory.getLogger(OutstandingRequestTracker.class); - - /** - * no requests; saves creating a new list if not needed - */ - private final List NO_REQUESTS = new ArrayList<>(0); - - private Map placedRequests = new HashMap<>(); - - /** - * List of open requests; no specific details on them. - */ - private List openRequests = new ArrayList<>(); - - /** - * Create a new request for the specific role. - *

- * If a location is set, the request is added to {@link #placedRequests}. - * If not, it is added to {@link #openRequests} - *

- * This does not update the node instance's role's request count - * @param instance node instance to manager - * @param role role index - * @return a new request - */ - public synchronized OutstandingRequest newRequest(NodeInstance instance, int role) { - OutstandingRequest request = new OutstandingRequest(role, instance); - if (request.isLocated()) { - placedRequests.put(request.getIndex(), request); - } else { - openRequests.add(request); - } - return request; - } - - /** - * Create a new Anti-affine request for the specific role - *

- * It is added to {@link #openRequests} - *

- * This does not update the node instance's role's request count - * @param role role index - * @param nodes list of suitable nodes - * @param label label to use - * @return a new request - */ - public synchronized OutstandingRequest newAARequest(int role, - List nodes, - String label) { - Preconditions.checkArgument(!nodes.isEmpty()); - // safety check to verify the allocation will hold - for (NodeInstance node : nodes) { - Preconditions.checkState(node.canHost(role, label), - "Cannot allocate role ID %d to node %s", role, node); - } - OutstandingRequest request = new OutstandingRequest(role, nodes); - openRequests.add(request); - return request; - } - - /** - * Look up any oustanding request to a (role, hostname). - * @param role role index - * @param hostname hostname - * @return the request or null if there was no outstanding one in the {@link #placedRequests} - */ - @VisibleForTesting - public synchronized OutstandingRequest lookupPlacedRequest(int role, String hostname) { - Preconditions.checkArgument(hostname != null, "null hostname"); - return placedRequests.get(new RoleHostnamePair(role, hostname)); - } - - /** - * Remove a request - * @param request matching request to find - * @return the request or null for no match in the {@link #placedRequests} - */ - @VisibleForTesting - public synchronized OutstandingRequest removePlacedRequest(OutstandingRequest request) { - return placedRequests.remove(request); - } - - /** - * Notification that a container has been allocated - * - *

    - *
  1. drop it from the {@link #placedRequests} structure.
  2. - *
  3. generate the cancellation request
  4. - *
  5. for AA placement, any actions needed
  6. - *
- * - * @param role role index - * @param hostname hostname - * @return the allocation outcome - */ - public synchronized ContainerAllocationResults onContainerAllocated(int role, - String hostname, - Container container) { - final String containerDetails = SliderUtils.containerToString(container); - log.debug("Processing allocation for role {} on {}", role, - containerDetails); - ContainerAllocationResults allocation = new ContainerAllocationResults(); - ContainerAllocationOutcome outcome; - OutstandingRequest request = placedRequests.remove(new OutstandingRequest(role, hostname)); - if (request != null) { - //satisfied request - log.debug("Found oustanding placed request for container: {}", request); - request.completed(); - // derive outcome from status of tracked request - outcome = request.isEscalated() - ? ContainerAllocationOutcome.Escalated - : ContainerAllocationOutcome.Placed; - } else { - // not in the list; this is an open placement - // scan through all containers in the open request list - request = removeOpenRequest(container); - if (request != null) { - log.debug("Found open outstanding request for container: {}", request); - request.completed(); - outcome = ContainerAllocationOutcome.Open; - } else { - log.warn("No oustanding request found for container {}, outstanding queue has {} entries ", - containerDetails, - openRequests.size()); - outcome = ContainerAllocationOutcome.Unallocated; - } - } - if (request != null && request.getIssuedRequest() != null) { - allocation.operations.add(request.createCancelOperation()); - } else { - // there's a request, but no idea what to cancel. - // rather than try to recover from it inelegantly, (and cause more confusion), - // log the event, but otherwise continue - log.warn("Unexpected allocation of container " + SliderUtils.containerToString(container)); - } - - allocation.origin = request; - allocation.outcome = outcome; - return allocation; - } - - /** - * Find and remove an open request. Determine it by scanning open requests - * for one whose priority & resource requirements match that of the container - * allocated. - * @param container container allocated - * @return a request which matches the allocation, or null for "no match" - */ - private OutstandingRequest removeOpenRequest(Container container) { - int pri = container.getPriority().getPriority(); - Resource resource = container.getResource(); - OutstandingRequest request = null; - ListIterator openlist = openRequests.listIterator(); - while (openlist.hasNext() && request == null) { - OutstandingRequest r = openlist.next(); - if (r.getPriority() == pri) { - // matching resource - if (r.resourceRequirementsMatch(resource)) { - // match of priority and resources - request = r; - openlist.remove(); - } else { - log.debug("Matched priorities but resources different"); - } - } - } - return request; - } - - /** - * Determine which host was a role type most recently used on, so that - * if a choice is made of which (potentially surplus) containers to use, - * the most recent one is picked first. This operation does not - * change the role history, though it queries it. - */ - static class newerThan implements Comparator { - private RoleHistory rh; - - public newerThan(RoleHistory rh) { - this.rh = rh; - } - - /** - * Get the age of a node hosting container. If it is not known in the history, - * return 0. - * @param c container - * @return age, null if there's no entry for it. - */ - private long getAgeOf(Container c) { - long age = 0; - NodeInstance node = rh.getExistingNodeInstance(c); - int role = ContainerPriority.extractRole(c); - if (node != null) { - NodeEntry nodeEntry = node.get(role); - if (nodeEntry != null) { - age = nodeEntry.getLastUsed(); - } - } - return age; - } - - /** - * Comparator: which host is more recent? - * @param c1 container 1 - * @param c2 container 2 - * @return 1 if c2 older-than c1, 0 if equal; -1 if c1 older-than c2 - */ - @Override - public int compare(Container c1, Container c2) { - int role1 = ContainerPriority.extractRole(c1); - int role2 = ContainerPriority.extractRole(c2); - if (role1 < role2) return -1; - if (role1 > role2) return 1; - - long age = getAgeOf(c1); - long age2 = getAgeOf(c2); - - if (age > age2) { - return -1; - } else if (age < age2) { - return 1; - } - // equal - return 0; - } - } - - /** - * Take a list of requests and split them into specific host requests and - * generic assignments. This is to give requested hosts priority - * in container assignments if more come back than expected - * @param rh RoleHistory instance - * @param inAllocated the list of allocated containers - * @param outPlaceRequested initially empty list of requested locations - * @param outUnplaced initially empty list of unrequested hosts - */ - public synchronized void partitionRequests(RoleHistory rh, - List inAllocated, - List outPlaceRequested, - List outUnplaced) { - Collections.sort(inAllocated, new newerThan(rh)); - for (Container container : inAllocated) { - int role = ContainerPriority.extractRole(container); - String hostname = RoleHistoryUtils.hostnameOf(container); - if (placedRequests.containsKey(new OutstandingRequest(role, hostname))) { - outPlaceRequested.add(container); - } else { - outUnplaced.add(container); - } - } - } - - - /** - * Reset list all outstanding requests for a role: return the hostnames - * of any canceled requests - * - * @param role role to cancel - * @return possibly empty list of hostnames - */ - public synchronized List resetOutstandingRequests(int role) { - List hosts = new ArrayList<>(); - Iterator> iterator = - placedRequests.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry next = - iterator.next(); - OutstandingRequest request = next.getValue(); - if (request.roleId == role) { - iterator.remove(); - request.completed(); - hosts.add(request.node); - } - } - ListIterator openlist = openRequests.listIterator(); - while (openlist.hasNext()) { - OutstandingRequest next = openlist.next(); - if (next.roleId == role) { - openlist.remove(); - } - } - return hosts; - } - - /** - * Get a list of outstanding requests. The list is cloned, but the contents - * are shared - * @return a list of the current outstanding requests - */ - public synchronized List listPlacedRequests() { - return new ArrayList<>(placedRequests.values()); - } - - /** - * Get a list of outstanding requests. The list is cloned, but the contents - * are shared - * @return a list of the current outstanding requests - */ - public synchronized List listOpenRequests() { - return new ArrayList<>(openRequests); - } - - /** - * Escalate operation as triggered by external timer. - * @return a (usually empty) list of cancel/request operations. - */ - @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") - public synchronized List escalateOutstandingRequests(long now) { - if (placedRequests.isEmpty()) { - return NO_REQUESTS; - } - - List operations = new ArrayList<>(); - for (OutstandingRequest outstandingRequest : placedRequests.values()) { - synchronized (outstandingRequest) { - // sync escalation check with operation so that nothing can happen to state - // of the request during the escalation - if (outstandingRequest.shouldEscalate(now)) { - - // time to escalate - CancelSingleRequest cancel = outstandingRequest.createCancelOperation(); - operations.add(cancel); - AMRMClient.ContainerRequest escalated = outstandingRequest.escalate(); - operations.add(new ContainerRequestOperation(escalated)); - } - } - - } - return operations; - } - - /** - * Cancel all outstanding AA requests from the lists of requests. - * - * This does not remove them from the role status; they must be reset - * by the caller. - * - */ - @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") - public synchronized List cancelOutstandingAARequests() { - - log.debug("Looking for AA request to cancel"); - List operations = new ArrayList<>(); - - // first, all placed requests - List requestsToRemove = new ArrayList<>(placedRequests.size()); - for (Map.Entry entry : placedRequests.entrySet()) { - OutstandingRequest outstandingRequest = entry.getValue(); - synchronized (outstandingRequest) { - if (outstandingRequest.isAntiAffine()) { - // time to escalate - operations.add(outstandingRequest.createCancelOperation()); - requestsToRemove.add(entry.getKey()); - } - } - } - for (RoleHostnamePair keys : requestsToRemove) { - placedRequests.remove(keys); - } - - // second, all open requests - ListIterator orit = openRequests.listIterator(); - while (orit.hasNext()) { - OutstandingRequest outstandingRequest = orit.next(); - synchronized (outstandingRequest) { - if (outstandingRequest.isAntiAffine()) { - // time to escalate - operations.add(outstandingRequest.createCancelOperation()); - orit.remove(); - } - } - } - log.info("Cancelling {} outstanding AA requests", operations.size()); - - return operations; - } - - /** - * Extract a specific number of open requests for a role - * @param roleId role Id - * @param count count to extract - * @return a list of requests which are no longer in the open request list - */ - public synchronized List extractOpenRequestsForRole(int roleId, int count) { - List results = new ArrayList<>(); - ListIterator openlist = openRequests.listIterator(); - while (openlist.hasNext() && count > 0) { - OutstandingRequest openRequest = openlist.next(); - if (openRequest.roleId == roleId) { - results.add(openRequest); - openlist.remove(); - count--; - } - } - return results; - } - - /** - * Extract a specific number of placed requests for a role - * @param roleId role Id - * @param count count to extract - * @return a list of requests which are no longer in the placed request data structure - */ - public synchronized List extractPlacedRequestsForRole(int roleId, int count) { - List results = new ArrayList<>(); - Iterator> - iterator = placedRequests.entrySet().iterator(); - while (iterator.hasNext() && count > 0) { - OutstandingRequest request = iterator.next().getValue(); - if (request.roleId == roleId) { - results.add(request); - count--; - } - } - // now cull them from the map - for (OutstandingRequest result : results) { - placedRequests.remove(result); - } - - return results; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java deleted file mode 100644 index 8fc08b7..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java +++ /dev/null @@ -1,277 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import com.google.common.cache.LoadingCache; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import org.apache.slider.api.ClusterNode; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.ConfigFile; -import org.apache.slider.api.types.ApplicationLivenessInformation; -import org.apache.slider.api.types.ComponentInformation; -import org.apache.slider.api.types.NodeInformation; -import org.apache.slider.api.types.RoleStatistics; -import org.apache.slider.core.exceptions.NoSuchNodeException; -import org.apache.slider.core.registry.docstore.PublishedConfigSet; -import org.apache.slider.core.registry.docstore.PublishedExportsSet; -import org.apache.slider.server.appmaster.web.rest.RestPaths; -import org.apache.slider.server.services.utility.PatternValidator; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -/** - * Implementation of {@link StateAccessForProviders}, which means - * state access for providers, web UI and IPC/REST views. - */ -public class ProviderAppState implements StateAccessForProviders { - - - private final Map publishedConfigSets = - new ConcurrentHashMap<>(5); - private final PublishedExportsSet publishedExportsSets = new PublishedExportsSet(); - private static final PatternValidator validator = new PatternValidator( - RestPaths.PUBLISHED_CONFIGURATION_SET_REGEXP); - private String applicationName; - - private final AppState appState; - - public ProviderAppState(String applicationName, AppState appState) { - this.appState = appState; - this.applicationName = applicationName; - } - - public void setApplicationName(String applicationName) { - this.applicationName = applicationName; - } - - @Override - public String getApplicationName() { - return applicationName; - } - - @Override - public PublishedConfigSet getPublishedSliderConfigurations() { - return getOrCreatePublishedConfigSet(RestPaths.SLIDER_CONFIGSET); - } - - @Override - public PublishedExportsSet getPublishedExportsSet() { - return publishedExportsSets; - } - - @Override - public PublishedConfigSet getPublishedConfigSet(String name) { - return publishedConfigSets.get(name); - } - - @Override - public PublishedConfigSet getOrCreatePublishedConfigSet(String name) { - PublishedConfigSet set = publishedConfigSets.get(name); - if (set == null) { - validator.validate(name); - synchronized (publishedConfigSets) { - // synchronized double check to ensure that there is never an overridden - // config set created - set = publishedConfigSets.get(name); - if (set == null) { - set = new PublishedConfigSet(); - publishedConfigSets.put(name, set); - } - } - } - return set; - } - - @Override - public List listConfigSets() { - - synchronized (publishedConfigSets) { - List sets = new ArrayList<>(publishedConfigSets.keySet()); - return sets; - } - } - - @Override - public Map getRoleStatusMap() { - return appState.getRoleStatusMap(); - } - - - @Override - public Map getFailedContainers() { - return appState.getFailedContainers(); - } - - @Override - public Map getLiveContainers() { - return appState.getLiveContainers(); - } - - @Override - public Application getApplication() { - return appState.getClusterStatus(); - } - - @Override - public boolean isApplicationLive() { - return appState.isApplicationLive(); - } - - @Override - public RoleStatus lookupRoleStatus(int key) { - return appState.lookupRoleStatus(key); - } - - @Override - public RoleStatus lookupRoleStatus(Container c) throws YarnRuntimeException { - return appState.lookupRoleStatus(c); - } - - @Override - public RoleStatus lookupRoleStatus(String name) throws YarnRuntimeException { - return appState.lookupRoleStatus(name); - } - - @Override - public List cloneOwnedContainerList() { - return appState.cloneOwnedContainerList(); - } - - @Override - public int getNumOwnedContainers() { - return appState.getNumOwnedContainers(); - } - - @Override - public RoleInstance getOwnedContainer(ContainerId id) { - return appState.getOwnedContainer(id); - } - - @Override - public RoleInstance getOwnedContainer(String id) throws NoSuchNodeException { - return appState.getOwnedInstanceByContainerID(id); - } - - @Override - public List cloneLiveContainerInfoList() { - return appState.cloneLiveContainerInfoList(); - } - - @Override - public RoleInstance getLiveInstanceByContainerID(String containerId) throws - NoSuchNodeException { - return appState.getLiveInstanceByContainerID(containerId); - } - - @Override - public List getLiveInstancesByContainerIDs(Collection containerIDs) { - return appState.getLiveInstancesByContainerIDs(containerIDs); - } - - @Override - public Application refreshClusterStatus() { - return appState.refreshClusterStatus(); - } - - @Override - public ApplicationLivenessInformation getApplicationLivenessInformation() { - return appState.getApplicationLivenessInformation(); - } - - @Override - public Map getComponentInfoSnapshot() { - return appState.getComponentInfoSnapshot(); - } - - @Override - public Map> getRoleClusterNodeMapping() { - return appState.createRoleToClusterNodeMap(); - } - - @Override - public List enumLiveInstancesInRole(String role) { - List nodes = new ArrayList<>(); - Collection allRoleInstances = cloneLiveContainerInfoList(); - for (RoleInstance node : allRoleInstances) { - if (role.isEmpty() || role.equals(node.role)) { - nodes.add(node); - } - } - return nodes; - } - - @Override - public List lookupRoleContainers(String component) { - RoleStatus roleStatus = lookupRoleStatus(component); - List ownedContainerList = cloneOwnedContainerList(); - List matching = new ArrayList<>(ownedContainerList.size()); - int roleId = roleStatus.getPriority(); - for (RoleInstance instance : ownedContainerList) { - if (instance.roleId == roleId) { - matching.add(instance); - } - } - return matching; - } - - @Override - public ComponentInformation getComponentInformation(String component) { - RoleStatus roleStatus = lookupRoleStatus(component); - ComponentInformation info = roleStatus.serialize(); - List containers = lookupRoleContainers(component); - info.containers = new ArrayList<>(containers.size()); - for (RoleInstance container : containers) { - info.containers.add(container.id); - } - return info; - } - - @Override - public Map getNodeInformationSnapshot() { - return appState.getRoleHistory() - .getNodeInformationSnapshot(appState.buildNamingMap()); - } - - @Override - public NodeInformation getNodeInformation(String hostname) { - return appState.getRoleHistory() - .getNodeInformation(hostname, appState.buildNamingMap()); - } - - @Override - public RoleStatistics getRoleStatistics() { - return appState.getRoleStatistics(); - } - - @Override - public Map getGlobalSubstitutionTokens() { - return appState.globalTokens; - } - - @Override - public LoadingCache getConfigFileCache() { - return appState.configFileCache; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java deleted file mode 100644 index 15333e4..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java +++ /dev/null @@ -1,1123 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.NodeState; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.slider.api.types.NodeInformation; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.providers.ProviderRole; -import org.apache.slider.server.appmaster.management.BoolMetric; -import org.apache.slider.server.appmaster.management.Timestamp; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.UpdateBlacklistOperation; -import org.apache.slider.server.avro.LoadedRoleHistory; -import org.apache.slider.server.avro.NodeEntryRecord; -import org.apache.slider.server.avro.RoleHistoryHeader; -import org.apache.slider.server.avro.RoleHistoryWriter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.concurrent.ConcurrentHashMap; - -/** - * The Role History. - *

- * Synchronization policy: all public operations are synchronized. - * Protected methods are in place for testing -no guarantees are made. - *

- * Inner classes have no synchronization guarantees; they should be manipulated - * in these classes and not externally. - *

- * Note that as well as some methods marked visible for testing, there - * is the option for the time generator method, {@link #now()} to - * be overridden so that a repeatable time series can be used. - * - */ -public class RoleHistory { - protected static final Logger log = - LoggerFactory.getLogger(RoleHistory.class); - private final List providerRoles; - /** the roles in here are shared with App State */ - private final Map roleStatusMap = new HashMap<>(); - private final AbstractClusterServices recordFactory; - - private long startTime; - - /** Time when saved */ - private final Timestamp saveTime = new Timestamp(0); - - /** If the history was loaded, the time at which the history was saved. - * That is: the time the data was valid */ - private final Timestamp thawedDataTime = new Timestamp(0); - - private NodeMap nodemap; - private int roleSize; - private final BoolMetric dirty = new BoolMetric(false); - private FileSystem filesystem; - private Path historyPath; - private RoleHistoryWriter historyWriter = new RoleHistoryWriter(); - - /** - * When were the nodes updated in a {@link #onNodesUpdated(List)} call? - * If zero: never. - */ - private final Timestamp nodesUpdatedTime = new Timestamp(0); - private final BoolMetric nodeUpdateReceived = new BoolMetric(false); - - private OutstandingRequestTracker outstandingRequests = - new OutstandingRequestTracker(); - - /** - * For each role, lists nodes that are available for data-local allocation, - * ordered by more recently released - to accelerate node selection. - * That is, they are "recently used nodes" - */ - private Map> recentNodes; - - /** - * Instantiate - * @param roles initial role list - * @param recordFactory yarn record factory - * @throws BadConfigException - */ - public RoleHistory(Collection roles, AbstractClusterServices recordFactory) throws BadConfigException { - this.recordFactory = recordFactory; - roleSize = roles.size(); - providerRoles = new ArrayList<>(roleSize); - for (RoleStatus role : roles) { - addNewRole(role); - } - reset(); - } - - /** - * Reset the variables -this does not adjust the fixed attributes - * of the history, but the nodemap and failed node map are cleared. - */ - protected synchronized void reset() throws BadConfigException { - - nodemap = new NodeMap(roleSize); - resetAvailableNodeLists(); - outstandingRequests = new OutstandingRequestTracker(); - } - - - /** - * safety check: make sure the role is unique amongst - * the role stats...which is extended with the new role - * @param roleStatus role - * @throws ArrayIndexOutOfBoundsException - * @throws BadConfigException - */ - protected void putRole(RoleStatus roleStatus) throws BadConfigException { - int index = roleStatus.getKey(); - if (index < 0) { - throw new BadConfigException("Provider " + roleStatus + " id is out of range"); - } - if (roleStatusMap.get(index) != null) { - throw new BadConfigException( - roleStatus.toString() + " id duplicates that of " + - roleStatusMap.get(index)); - } - roleStatusMap.put(index, roleStatus); - } - - /** - * Add a new role - * @param roleStatus new role - */ - public void addNewRole(RoleStatus roleStatus) throws BadConfigException { - log.debug("Validating/adding new role to role history: {} ", roleStatus); - putRole(roleStatus); - this.providerRoles.add(roleStatus.getProviderRole()); - } - - /** - * Lookup a role by ID - * @param roleId role Id - * @return role or null if not found - */ - public ProviderRole lookupRole(int roleId) { - for (ProviderRole role : providerRoles) { - if (role.id == roleId) { - return role; - } - } - return null; - } - - /** - * Clear the lists of available nodes - */ - private synchronized void resetAvailableNodeLists() { - recentNodes = new ConcurrentHashMap<>(roleSize); - } - - /** - * Prepare the history for re-reading its state. - *

- * This intended for use by the RoleWriter logic. - * @throws BadConfigException if there is a problem rebuilding the state - */ - private void prepareForReading(RoleHistoryHeader header) - throws BadConfigException { - reset(); - - int roleCountInSource = header.getRoles(); - if (roleCountInSource != roleSize) { - log.warn("Number of roles in source {}" - +" does not match the expected number of {}", - roleCountInSource, - roleSize); - } - //record when the data was loaded - setThawedDataTime(header.getSaved()); - } - - /** - * rebuild the placement history from the loaded role history - * @param loadedRoleHistory loaded history - * @return the number of entries discarded - * @throws BadConfigException if there is a problem rebuilding the state - */ - @VisibleForTesting - public synchronized int rebuild(LoadedRoleHistory loadedRoleHistory) throws BadConfigException { - RoleHistoryHeader header = loadedRoleHistory.getHeader(); - prepareForReading(header); - int discarded = 0; - Long saved = header.getSaved(); - for (NodeEntryRecord nodeEntryRecord : loadedRoleHistory.records) { - Integer roleId = nodeEntryRecord.getRole(); - NodeEntry nodeEntry = new NodeEntry(roleId); - nodeEntry.setLastUsed(nodeEntryRecord.getLastUsed()); - if (nodeEntryRecord.getActive()) { - //if active at the time of save, make the last used time the save time - nodeEntry.setLastUsed(saved); - } - String hostname = SliderUtils.sequenceToString(nodeEntryRecord.getHost()); - ProviderRole providerRole = lookupRole(roleId); - if (providerRole == null) { - // discarding entry - log.info("Discarding history entry with unknown role: {} on host {}", - roleId, hostname); - discarded ++; - } else { - NodeInstance instance = getOrCreateNodeInstance(hostname); - instance.set(roleId, nodeEntry); - } - } - return discarded; - } - - public synchronized long getStartTime() { - return startTime; - } - - public synchronized long getSaveTime() { - return saveTime.get(); - } - - public long getThawedDataTime() { - return thawedDataTime.get(); - } - - public void setThawedDataTime(long thawedDataTime) { - this.thawedDataTime.set(thawedDataTime); - } - - public synchronized int getRoleSize() { - return roleSize; - } - - /** - * Get the total size of the cluster -the number of NodeInstances - * @return a count - */ - public synchronized int getClusterSize() { - return nodemap.size(); - } - - public synchronized boolean isDirty() { - return dirty.get(); - } - - public synchronized void setDirty(boolean dirty) { - this.dirty.set(dirty); - } - - /** - * Tell the history that it has been saved; marks itself as clean - * @param timestamp timestamp -updates the savetime field - */ - public synchronized void saved(long timestamp) { - setDirty(false); - saveTime.set(timestamp); - } - - /** - * Get a clone of the nodemap. - * The instances inside are not cloned - * @return the map - */ - public synchronized NodeMap cloneNodemap() { - return (NodeMap) nodemap.clone(); - } - - /** - * Get snapshot of the node map - * @return a snapshot of the current node state - * @param naming naming map of priority to enty name; entries must be unique. - * It's OK to be incomplete, for those the list falls back to numbers. - */ - public synchronized Map getNodeInformationSnapshot( - Map naming) { - Map result = new HashMap<>(nodemap.size()); - for (Map.Entry entry : nodemap.entrySet()) { - result.put(entry.getKey(), entry.getValue().serialize(naming)); - } - return result; - } - - /** - * Get the information on a node - * @param hostname hostname - * @param naming naming map of priority to enty name; entries must be unique. - * It's OK to be incomplete, for those the list falls back to numbers. - * @return the information about that host, or null if there is none - */ - public synchronized NodeInformation getNodeInformation(String hostname, - Map naming) { - NodeInstance nodeInstance = nodemap.get(hostname); - return nodeInstance != null ? nodeInstance.serialize(naming) : null; - } - - /** - * Get the node instance for the specific node -creating it if needed - * @param hostname node address - * @return the instance - */ - public synchronized NodeInstance getOrCreateNodeInstance(String hostname) { - //convert to a string - return nodemap.getOrCreate(hostname); - } - - /** - * Insert a list of nodes into the map; overwrite any with that name. - * This is a bulk operation for testing. - * Important: this does not update the available node lists, these - * must be rebuilt afterwards. - * @param nodes collection of nodes. - */ - @VisibleForTesting - public synchronized void insert(Collection nodes) { - nodemap.insert(nodes); - } - - /** - * Get current time. overrideable for test subclasses - * @return current time in millis - */ - protected long now() { - return System.currentTimeMillis(); - } - - /** - * Mark ourselves as dirty - */ - public void touch() { - setDirty(true); - try { - saveHistoryIfDirty(); - } catch (IOException e) { - log.warn("Failed to save history file ", e); - } - } - - /** - * reset the failed recently counters - */ - public synchronized void resetFailedRecently() { - log.info("Resetting failure history"); - nodemap.resetFailedRecently(); - } - - /** - * Get the path used for history files - * @return the directory used for history files - */ - public Path getHistoryPath() { - return historyPath; - } - - /** - * Save the history to its location using the timestamp as part of - * the filename. The saveTime and dirty fields are updated - * @param time timestamp timestamp to use as the save time - * @return the path saved to - * @throws IOException IO problems - */ - @VisibleForTesting - public synchronized Path saveHistory(long time) throws IOException { - Path filename = historyWriter.createHistoryFilename(historyPath, time); - historyWriter.write(filesystem, filename, true, this, time); - saved(time); - return filename; - } - - /** - * Save the history with the current timestamp if it is dirty; - * return the path saved to if this is the case - * @return the path or null if the history was not saved - * @throws IOException failed to save for some reason - */ - public synchronized Path saveHistoryIfDirty() throws IOException { - if (isDirty()) { - return saveHistory(now()); - } else { - return null; - } - } - - /** - * Start up - * @param fs filesystem - * @param historyDir path in FS for history - * @return true if the history was thawed - */ - public boolean onStart(FileSystem fs, Path historyDir) throws BadConfigException { - assert filesystem == null; - filesystem = fs; - historyPath = historyDir; - startTime = now(); - //assume the history is being thawed; this will downgrade as appropriate - return onThaw(); - } - - /** - * Handler for bootstrap event: there was no history to thaw - */ - public void onBootstrap() { - log.debug("Role history bootstrapped"); - } - - /** - * Handle the start process after the history has been rebuilt, - * and after any gc/purge - */ - public synchronized boolean onThaw() throws BadConfigException { - assert filesystem != null; - assert historyPath != null; - boolean thawSuccessful = false; - //load in files from data dir - - LoadedRoleHistory loadedRoleHistory = null; - try { - loadedRoleHistory = historyWriter.loadFromHistoryDir(filesystem, historyPath); - } catch (IOException e) { - log.warn("Exception trying to load history from {}", historyPath, e); - } - if (loadedRoleHistory != null) { - rebuild(loadedRoleHistory); - thawSuccessful = true; - Path loadPath = loadedRoleHistory.getPath(); - log.debug("loaded history from {}", loadPath); - // delete any old entries - try { - int count = historyWriter.purgeOlderHistoryEntries(filesystem, loadPath); - log.debug("Deleted {} old history entries", count); - } catch (IOException e) { - log.info("Ignoring exception raised while trying to delete old entries", - e); - } - - //start is then completed - buildRecentNodeLists(); - } else { - //fallback to bootstrap procedure - onBootstrap(); - } - return thawSuccessful; - } - - - /** - * (After the start), rebuild the availability data structures - */ - @VisibleForTesting - public synchronized void buildRecentNodeLists() { - resetAvailableNodeLists(); - // build the list of available nodes - for (Map.Entry entry : nodemap.entrySet()) { - NodeInstance ni = entry.getValue(); - for (int i = 0; i < roleSize; i++) { - NodeEntry nodeEntry = ni.get(i); - if (nodeEntry != null && nodeEntry.isAvailable()) { - log.debug("Adding {} for role {}", ni, i); - listRecentNodesForRoleId(i).add(ni); - } - } - } - // sort the resulting arrays - for (int i = 0; i < roleSize; i++) { - sortRecentNodeList(i); - } - } - - /** - * Get the nodes for an ID -may be null - * @param id role ID - * @return potentially null list - */ - @VisibleForTesting - public List getRecentNodesForRoleId(int id) { - return recentNodes.get(id); - } - - /** - * Get a possibly empty list of suggested nodes for a role. - * @param id role ID - * @return list - */ - private LinkedList listRecentNodesForRoleId(int id) { - LinkedList instances = recentNodes.get(id); - if (instances == null) { - synchronized (this) { - // recheck in the synchronized block and recreate - if (recentNodes.get(id) == null) { - recentNodes.put(id, new LinkedList()); - } - instances = recentNodes.get(id); - } - } - return instances; - } - - /** - * Sort a the recent node list for a single role - * @param role role to sort - */ - private void sortRecentNodeList(int role) { - List nodesForRoleId = getRecentNodesForRoleId(role); - if (nodesForRoleId != null) { - Collections.sort(nodesForRoleId, new NodeInstance.Preferred(role)); - } - } - - public synchronized UpdateBlacklistOperation updateBlacklist( - Collection roleStatuses) { - List blacklistAdditions = new ArrayList<>(); - List blacklistRemovals = new ArrayList<>(); - for (Entry nodeInstanceEntry : nodemap.entrySet()) { - boolean shouldBeBlacklisted = false; - String nodeHost = nodeInstanceEntry.getKey(); - NodeInstance nodeInstance = nodeInstanceEntry.getValue(); - for (RoleStatus roleStatus : roleStatuses) { - if (nodeInstance.exceedsFailureThreshold(roleStatus)) { - shouldBeBlacklisted = true; - break; - } - } - if (shouldBeBlacklisted) { - if (!nodeInstance.isBlacklisted()) { - blacklistAdditions.add(nodeHost); - nodeInstance.setBlacklisted(true); - } - } else { - if (nodeInstance.isBlacklisted()) { - blacklistRemovals.add(nodeHost); - nodeInstance.setBlacklisted(false); - } - } - } - if (blacklistAdditions.isEmpty() && blacklistRemovals.isEmpty()) { - return null; - } - return new UpdateBlacklistOperation(blacklistAdditions, blacklistRemovals); - } - - /** - * Find a node for use - * @param role role - * @return the instance, or null for none - */ - @VisibleForTesting - public synchronized NodeInstance findRecentNodeForNewInstance(RoleStatus role) { - if (!role.isPlacementDesired()) { - // no data locality policy - return null; - } - int roleId = role.getKey(); - boolean strictPlacement = role.isStrictPlacement(); - NodeInstance nodeInstance = null; - // Get the list of possible targets. - // This is a live list: changes here are preserved - List targets = getRecentNodesForRoleId(roleId); - if (targets == null) { - // nothing to allocate on - return null; - } - - int cnt = targets.size(); - log.debug("There are {} node(s) to consider for {}", cnt, role.getName()); - for (int i = 0; i < cnt && nodeInstance == null; i++) { - NodeInstance candidate = targets.get(i); - if (candidate.getActiveRoleInstances(roleId) == 0) { - // no active instances: check failure statistics - if (strictPlacement - || (candidate.isOnline() && !candidate.exceedsFailureThreshold(role))) { - targets.remove(i); - // exit criteria for loop is now met - nodeInstance = candidate; - } else { - // too many failures for this node - log.info("Recent node failures is higher than threshold {}. Not requesting host {}", - role.getNodeFailureThreshold(), candidate.hostname); - } - } - } - - if (nodeInstance == null) { - log.info("No node found for {}", role.getName()); - } - return nodeInstance; - } - - /** - * Find a node for use - * @param role role - * @return the instance, or null for none - */ - @VisibleForTesting - public synchronized List findNodeForNewAAInstance(RoleStatus role) { - // all nodes that are live and can host the role; no attempt to exclude ones - // considered failing - return nodemap.findAllNodesForRole(role.getKey(), role.getLabelExpression()); - } - - /** - * Request an instance on a given node. - * An outstanding request is created & tracked, with the - * relevant node entry for that role updated. - *

- * The role status entries will also be tracked - *

- * Returns the request that is now being tracked. - * If the node instance is not null, it's details about the role is incremented - * - * @param node node to target or null for "any" - * @param role role to request - * @return the request - */ - public synchronized OutstandingRequest requestInstanceOnNode( - NodeInstance node, RoleStatus role, Resource resource) { - OutstandingRequest outstanding = outstandingRequests.newRequest(node, role.getKey()); - outstanding.buildContainerRequest(resource, role, now()); - return outstanding; - } - - /** - * Find a node for a role and request an instance on that (or a location-less - * instance) - * @param role role status - * @return a request ready to go, or null if this is an AA request and no - * location can be found. - */ - public synchronized OutstandingRequest requestContainerForRole(RoleStatus role) { - - if (role.isAntiAffinePlacement()) { - return requestContainerForAARole(role); - } else { - Resource resource = recordFactory.newResource(); - role.copyResourceRequirements(resource); - NodeInstance node = findRecentNodeForNewInstance(role); - return requestInstanceOnNode(node, role, resource); - } - } - - /** - * Find a node for an AA role and request an instance on that (or a location-less - * instance) - * @param role role status - * @return a request ready to go, or null if no location can be found. - */ - public synchronized OutstandingRequest requestContainerForAARole(RoleStatus role) { - List nodes = findNodeForNewAAInstance(role); - if (!nodes.isEmpty()) { - OutstandingRequest outstanding = outstandingRequests.newAARequest( - role.getKey(), nodes, role.getLabelExpression()); - Resource resource = recordFactory.newResource(); - role.copyResourceRequirements(resource); - outstanding.buildContainerRequest(resource, role, now()); - return outstanding; - } else { - log.warn("No suitable location for {}", role.getName()); - return null; - } - } - /** - * Get the list of active nodes ... walks the node map so - * is {@code O(nodes)} - * @param role role index - * @return a possibly empty list of nodes with an instance of that node - */ - public synchronized List listActiveNodes(int role) { - return nodemap.listActiveNodes(role); - } - - /** - * Get the node entry of a container - * @param container container to look up - * @return the entry - * @throws RuntimeException if the container has no hostname - */ - public NodeEntry getOrCreateNodeEntry(Container container) { - return getOrCreateNodeInstance(container).getOrCreate(container); - } - - /** - * Get the node instance of a container -always returns something - * @param container container to look up - * @return a (possibly new) node instance - * @throws RuntimeException if the container has no hostname - */ - public synchronized NodeInstance getOrCreateNodeInstance(Container container) { - return nodemap.getOrCreate(RoleHistoryUtils.hostnameOf(container)); - } - - /** - * Get the node instance of a host if defined - * @param hostname hostname to look up - * @return a node instance or null - * @throws RuntimeException if the container has no hostname - */ - public synchronized NodeInstance getExistingNodeInstance(String hostname) { - return nodemap.get(hostname); - } - - /** - * Get the node instance of a container if there's an entry in the history - * @param container container to look up - * @return a node instance or null - * @throws RuntimeException if the container has no hostname - */ - public synchronized NodeInstance getExistingNodeInstance(Container container) { - return nodemap.get(RoleHistoryUtils.hostnameOf(container)); - } - - /** - * Perform any pre-allocation operations on the list of allocated containers - * based on knowledge of system state. - * Currently this places requested hosts ahead of unrequested ones. - * @param allocatedContainers list of allocated containers - * @return list of containers potentially reordered - */ - public synchronized List prepareAllocationList(List allocatedContainers) { - - //partition into requested and unrequested - List requested = - new ArrayList<>(allocatedContainers.size()); - List unrequested = - new ArrayList<>(allocatedContainers.size()); - outstandingRequests.partitionRequests(this, allocatedContainers, requested, unrequested); - - //give the unrequested ones lower priority - requested.addAll(unrequested); - return requested; - } - - /** - * A container has been allocated on a node -update the data structures - * @param container container - * @param desiredCount desired #of instances - * @param actualCount current count of instances - * @return The allocation outcome - */ - public synchronized ContainerAllocationResults onContainerAllocated(Container container, - long desiredCount, - long actualCount) { - int role = ContainerPriority.extractRole(container); - - String hostname = RoleHistoryUtils.hostnameOf(container); - List nodeInstances = listRecentNodesForRoleId(role); - ContainerAllocationResults outcome = - outstandingRequests.onContainerAllocated(role, hostname, container); - if (desiredCount <= actualCount) { - // all outstanding requests have been satisfied - // clear all the lists, so returning nodes to the available set - List hosts = outstandingRequests.resetOutstandingRequests(role); - if (!hosts.isEmpty()) { - //add the list - log.info("Adding {} hosts for role {}", hosts.size(), role); - nodeInstances.addAll(hosts); - sortRecentNodeList(role); - } - } - return outcome; - } - - /** - * A container has been assigned to a role instance on a node -update the data structures - * @param container container - */ - public void onContainerAssigned(Container container) { - NodeInstance node = getOrCreateNodeInstance(container); - NodeEntry nodeEntry = node.getOrCreate(container); - nodeEntry.onStarting(); - log.debug("Node {} has updated NodeEntry {}", node, nodeEntry); - } - - /** - * Event: a container start has been submitted - * @param container container being started - * @param instance instance bound to the container - */ - public void onContainerStartSubmitted(Container container, - RoleInstance instance) { - // no actions here - } - - /** - * Container start event - * @param container container that just started - */ - public void onContainerStarted(Container container) { - NodeEntry nodeEntry = getOrCreateNodeEntry(container); - nodeEntry.onStartCompleted(); - touch(); - } - - /** - * A container failed to start: update the node entry state - * and return the container to the queue - * @param container container that failed - * @return true if the node was queued - */ - public boolean onNodeManagerContainerStartFailed(Container container) { - return markContainerFinished(container, false, true, ContainerOutcome.Failed); - } - - /** - * Does the RoleHistory have enough information about the YARN cluster - * to start placing AA requests? That is: has it the node map and - * any label information needed? - * @return true if the caller can start requesting AA nodes - */ - public boolean canPlaceAANodes() { - return nodeUpdateReceived.get(); - } - - /** - * Get the last time the nodes were updated from YARN - * @return the update time or zero if never updated. - */ - public long getNodesUpdatedTime() { - return nodesUpdatedTime.get(); - } - - /** - * Update failedNodes and nodemap based on the node state - * - * @param updatedNodes list of updated nodes - * @return true if a review should be triggered. - */ - public synchronized boolean onNodesUpdated(List updatedNodes) { - log.debug("Updating {} nodes", updatedNodes.size()); - nodesUpdatedTime.set(now()); - nodeUpdateReceived.set(true); - int printed = 0; - boolean triggerReview = false; - for (NodeReport updatedNode : updatedNodes) { - String hostname = updatedNode.getNodeId() == null - ? "" - : updatedNode.getNodeId().getHost(); - NodeState nodeState = updatedNode.getNodeState(); - if (hostname.isEmpty() || nodeState == null) { - log.warn("Ignoring incomplete update"); - continue; - } - if (log.isDebugEnabled() && printed++ < 10) { - // log the first few, but avoid overloading the logs for a full cluster - // update - log.debug("Node \"{}\" is in state {}", hostname, nodeState); - } - // update the node; this also creates an instance if needed - boolean updated = nodemap.updateNode(hostname, updatedNode); - triggerReview |= updated; - } - return triggerReview; - } - - /** - * A container release request was issued - * @param container container submitted - */ - public void onContainerReleaseSubmitted(Container container) { - NodeEntry nodeEntry = getOrCreateNodeEntry(container); - nodeEntry.release(); - } - - /** - * App state notified of a container completed - * @param container completed container - * @return true if the node was queued - */ - public boolean onReleaseCompleted(Container container) { - return markContainerFinished(container, true, false, ContainerOutcome.Failed); - } - - /** - * App state notified of a container completed -but as - * it wasn't being released it is marked as failed - * - * @param container completed container - * @param shortLived was the container short lived? - * @param outcome - * @return true if the node is considered available for work - */ - public boolean onFailedContainer(Container container, - boolean shortLived, - ContainerOutcome outcome) { - return markContainerFinished(container, false, shortLived, outcome); - } - - /** - * Mark a container finished; if it was released then that is treated - * differently. history is {@code touch()}-ed - * - * - * @param container completed container - * @param wasReleased was the container released? - * @param shortLived was the container short lived? - * @param outcome - * @return true if the node was queued - */ - protected synchronized boolean markContainerFinished(Container container, - boolean wasReleased, - boolean shortLived, - ContainerOutcome outcome) { - NodeEntry nodeEntry = getOrCreateNodeEntry(container); - log.info("Finished container for node {}, released={}, shortlived={}", - nodeEntry.rolePriority, wasReleased, shortLived); - boolean available; - if (shortLived) { - nodeEntry.onStartFailed(); - available = false; - } else { - available = nodeEntry.containerCompleted(wasReleased, outcome); - maybeQueueNodeForWork(container, nodeEntry, available); - } - touch(); - return available; - } - - /** - * If the node is marked as available; queue it for assignments. - * Unsynced: requires caller to be in a sync block. - * @param container completed container - * @param nodeEntry node - * @param available available flag - * @return true if the node was queued - */ - private boolean maybeQueueNodeForWork(Container container, - NodeEntry nodeEntry, - boolean available) { - if (available) { - //node is free - nodeEntry.setLastUsed(now()); - NodeInstance ni = getOrCreateNodeInstance(container); - int roleId = ContainerPriority.extractRole(container); - log.debug("Node {} is now available for role id {}", ni, roleId); - listRecentNodesForRoleId(roleId).addFirst(ni); - } - return available; - } - - /** - * Print the history to the log. This is for testing and diagnostics - */ - public synchronized void dump() { - for (ProviderRole role : providerRoles) { - log.info(role.toString()); - List instances = listRecentNodesForRoleId(role.id); - log.info(" available: " + instances.size() - + " " + SliderUtils.joinWithInnerSeparator(" ", instances)); - } - - log.info("Nodes in Cluster: {}", getClusterSize()); - for (NodeInstance node : nodemap.values()) { - log.info(node.toFullString()); - } - } - - /** - * Build the mapping entry for persisting to the role history - * @return a mapping object - */ - public synchronized Map buildMappingForHistoryFile() { - Map mapping = new HashMap<>(getRoleSize()); - for (ProviderRole role : providerRoles) { - mapping.put(role.name, role.id); - } - return mapping; - } - - /** - * Get a clone of the available list - * @param role role index - * @return a clone of the list - */ - @VisibleForTesting - public List cloneRecentNodeList(int role) { - return new LinkedList<>(listRecentNodesForRoleId(role)); - } - - /** - * Get a snapshot of the outstanding placed request list - * @return a list of the requests outstanding at the time of requesting - */ - @VisibleForTesting - public List listPlacedRequests() { - return outstandingRequests.listPlacedRequests(); - } - - /** - * Get a snapshot of the outstanding placed request list - * @return a list of the requests outstanding at the time of requesting - */ - @VisibleForTesting - public List listOpenRequests() { - return outstandingRequests.listOpenRequests(); - } - - /** - * Escalate operation as triggered by external timer. - * @return a (usually empty) list of cancel/request operations. - */ - public synchronized List escalateOutstandingRequests() { - return outstandingRequests.escalateOutstandingRequests(now()); - } - /** - * Escalate operation as triggered by external timer. - * @return a (usually empty) list of cancel/request operations. - */ - public List cancelOutstandingAARequests() { - return outstandingRequests.cancelOutstandingAARequests(); - } - - /** - * Cancel a number of outstanding requests for a role -that is, not - * actual containers, just requests for new ones. - * @param role role - * @param toCancel number to cancel - * @return a list of cancellable operations. - */ - public List cancelRequestsForRole(RoleStatus role, int toCancel) { - return role.isAntiAffinePlacement() ? - cancelRequestsForAARole(role, toCancel) - : cancelRequestsForSimpleRole(role, toCancel); - } - - /** - * Build the list of requests to cancel from the outstanding list. - * @param role role - * @param toCancel number to cancel - * @return a list of cancellable operations. - */ - private synchronized List cancelRequestsForSimpleRole(RoleStatus role, int toCancel) { - Preconditions.checkArgument(toCancel > 0, - "trying to cancel invalid number of requests: " + toCancel); - List results = new ArrayList<>(toCancel); - // first scan through the unplaced request list to find all of a role - int roleId = role.getKey(); - List requests = - outstandingRequests.extractOpenRequestsForRole(roleId, toCancel); - - // are there any left? - int remaining = toCancel - requests.size(); - // ask for some placed nodes - requests.addAll(outstandingRequests.extractPlacedRequestsForRole(roleId, remaining)); - - // build cancellations - for (OutstandingRequest request : requests) { - results.add(request.createCancelOperation()); - } - return results; - } - - /** - * Build the list of requests to cancel for an AA role. This reduces the number - * of outstanding pending requests first, then cancels any active request, - * before finally asking for any placed containers - * @param role role - * @param toCancel number to cancel - * @return a list of cancellable operations. - */ - private synchronized List cancelRequestsForAARole(RoleStatus role, int toCancel) { - List results = new ArrayList<>(toCancel); - int roleId = role.getKey(); - List requests = new ArrayList<>(toCancel); - // there may be pending requests which can be cancelled here - long pending = role.getAAPending(); - if (pending > 0) { - // there are some pending ones which can be cancelled first - long pendingToCancel = Math.min(pending, toCancel); - log.info("Cancelling {} pending AA allocations, leaving {}", toCancel, - pendingToCancel); - role.setAAPending(pending - pendingToCancel); - toCancel -= pendingToCancel; - } - if (toCancel > 0 && role.isAARequestOutstanding()) { - // not enough - log.info("Cancelling current AA request"); - // find the single entry which may be running - requests = outstandingRequests.extractOpenRequestsForRole(roleId, toCancel); - role.cancelOutstandingAARequest(); - toCancel--; - } - - // ask for some excess nodes - if (toCancel > 0) { - requests.addAll(outstandingRequests.extractPlacedRequestsForRole(roleId, toCancel)); - } - - // build cancellations - for (OutstandingRequest request : requests) { - results.add(request.createCancelOperation()); - } - return results; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistoryUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistoryUtils.java deleted file mode 100644 index ea6197b..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistoryUtils.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.slider.common.tools.SliderUtils; - -public class RoleHistoryUtils { - - public static String hostnameOf(Container container) { - NodeId nodeId = container.getNodeId(); - if (nodeId== null) { - throw new RuntimeException("Container has no node ID: %s" + - SliderUtils.containerToString(container)); - } - return nodeId.getHost(); - } - - /** - * Decrement a value but hold it at zero. Usually a sanity check - * on counters tracking outstanding operations - * @param val value - * @return decremented value - */ - public static int decToFloor(int val) { - int v = val-1; - if (v < 0) { - v = 0; - } - return v; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHostnamePair.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHostnamePair.java deleted file mode 100644 index 920887a..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHostnamePair.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import java.util.Objects; - -public class RoleHostnamePair { - - /** - * requested role - */ - public final int roleId; - - /** - * hostname -will be null if node==null - */ - public final String hostname; - - public RoleHostnamePair(int roleId, String hostname) { - this.roleId = roleId; - this.hostname = hostname; - } - - public int getRoleId() { - return roleId; - } - - public String getHostname() { - return hostname; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof RoleHostnamePair)) { - return false; - } - RoleHostnamePair that = (RoleHostnamePair) o; - return Objects.equals(roleId, that.roleId) && - Objects.equals(hostname, that.hostname); - } - - @Override - public int hashCode() { - return Objects.hash(roleId, hostname); - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder( - "RoleHostnamePair{"); - sb.append("roleId=").append(roleId); - sb.append(", hostname='").append(hostname).append('\''); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleInstance.java deleted file mode 100644 index 070f2f8..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleInstance.java +++ /dev/null @@ -1,343 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.registry.client.binding.RegistryTypeUtils; -import org.apache.hadoop.registry.client.types.Endpoint; -import org.apache.hadoop.registry.client.types.ProtocolTypes; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.slider.api.ClusterNode; -import org.apache.slider.api.proto.Messages; -import org.apache.slider.api.types.ContainerInformation; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.providers.ProviderRole; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -/** - * Tracking information about a container - */ -public final class RoleInstance implements Cloneable { - - public Container container; - public ProviderRole providerRole; - public long componentId = -1; - public String compInstanceName = null; - /** - * Container ID - */ - public final String id; - public long createTime; - public long startTime; - /** - * flag set when it is released, to know if it has - * already been targeted for termination - */ - public boolean released; - - /** - * Name of the role - */ - public String role; - - /** - * Version of the app - */ - public String appVersion; - - /** - * Role Id; matches priority in resources.json - */ - public int roleId; - - /** - * state from StateValues - */ - public int state; - - /** - * Exit code: only valid if the state >= STOPPED - */ - public int exitCode; - - /** - * what was the command executed? - */ - public String command; - - /** - * Any diagnostics - */ - public String diagnostics; - - /** - * What is the tail output from the executed process (or [] if not started - * or the log cannot be picked up - */ - public String[] output; - - /** - * Any environment details - */ - public String[] environment; - - public String ip; - public String hostname; - public String host; - public String hostURL; - public ContainerAllocationOutcome placement; - public Path compInstanceDir; - - /** - * A list of registered endpoints. - */ - private List endpoints = - new ArrayList<>(2); - - public RoleInstance(Container container, ProviderRole role) { - this(container); - if (role.componentIdCounter != null) { - componentId = role.componentIdCounter.getAndIncrement(); - compInstanceName = role.name + componentId; - } else { - compInstanceName = role.name; - } - compInstanceName = compInstanceName.toLowerCase().replaceAll("_", "-"); - this.providerRole = role; - } - - public RoleInstance(Container container, RoleInstance failedInstance) { - this(container); - this.componentId = failedInstance.componentId; - this.compInstanceName = failedInstance.compInstanceName; - this.providerRole = failedInstance.providerRole; - } - - /** - * Create an instance to track an allocated container - * @param container a container which must be non null, and have a non-null Id field. - */ - public RoleInstance(Container container) { - Preconditions.checkNotNull(container, "Null container"); - Preconditions.checkState(container.getId() != null, - "Null container ID"); - - this.container = container; - id = container.getId().toString(); - if (container.getNodeId() != null) { - host = container.getNodeId().getHost(); - } - if (container.getNodeHttpAddress() != null) { - hostURL = "http://" + container.getNodeHttpAddress(); - } - } - - public NodeId getHost() { - return container.getNodeId(); - } - - @Override - public String toString() { - final StringBuilder sb = - new StringBuilder("RoleInstance{"); - sb.append("role='").append(role).append('\''); - sb.append(", id='").append(id).append('\''); - sb.append(", instanceName='").append(compInstanceName).append('\''); - sb.append(", container=").append(SliderUtils.containerToString(container)); - sb.append(", createTime=").append(createTime); - sb.append(", startTime=").append(startTime); - sb.append(", released=").append(released); - sb.append(", roleId=").append(roleId); - sb.append(", host=").append(host); - sb.append(", hostURL=").append(hostURL); - sb.append(", state=").append(state); - sb.append(", placement=").append(placement); - sb.append(", exitCode=").append(exitCode); - sb.append(", command='").append(command).append('\''); - sb.append(", diagnostics='").append(diagnostics).append('\''); - sb.append(", output=").append(Arrays.toString(output)); - sb.append(", environment=").append(Arrays.toString(environment)); - sb.append('}'); - return sb.toString(); - } - - public ContainerId getContainerId() { - return container.getId(); - } - - /** - * Generate the protobuf format of a request - * @return protobuf format. This excludes the Container info - */ - public Messages.RoleInstanceState toProtobuf() { - Messages.RoleInstanceState.Builder builder = - Messages.RoleInstanceState.newBuilder(); - if (container != null) { - builder.setName(container.getId().toString()); - } else { - builder.setName("unallocated instance"); - } - if (command != null) { - builder.setCommand(command); - } - if (environment != null) { - builder.addAllEnvironment(Arrays.asList(environment)); - } - if (diagnostics != null) { - builder.setDiagnostics(diagnostics); - } - builder.setExitCode(exitCode); - - if (output != null) { - builder.addAllOutput(Arrays.asList(output)); - } - if (role != null) { - builder.setRole(role); - } - builder.setRoleId(roleId); - builder.setState(state); - - builder.setReleased(released); - builder.setCreateTime(createTime); - builder.setStartTime(startTime); - builder.setHost(host); - builder.setHostURL(hostURL); - if (appVersion != null) { - builder.setAppVersion(appVersion); - } - return builder.build(); - } - - /** - * Build a serializable ClusterNode structure from this instance. - * This operation is unsynchronized. - * @return a serialized value. - */ - public ClusterNode toClusterNode() { - ClusterNode node; - if (container != null) { - node = new ClusterNode(container.getId()); - } else { - node = new ClusterNode(); - node.name = "unallocated instance"; - } - node.command = command; - node.createTime = createTime; - node.diagnostics = diagnostics; - if (environment != null) { - node.environment = Arrays.copyOf(environment, environment.length); - } - node.exitCode = exitCode; - node.ip = ip; - node.hostname = hostname; - node.host = host; - node.hostUrl = hostURL; - if (output != null) { - node.output = Arrays.copyOf(output, output.length); - } - node.released = released; - node.role = role; - node.roleId = roleId; - node.startTime = startTime ; - node.state = state; - - return node; - } - - /** - * Clone operation clones all the simple values but shares the - * Container object into the cloned copy -same with the output, - * diagnostics and env arrays. - * @return a clone of the object - * @throws CloneNotSupportedException - */ - @Override - public Object clone() throws CloneNotSupportedException { - RoleInstance cloned = (RoleInstance) super.clone(); - // clone the endpoint list, but not the values - cloned.endpoints = new ArrayList(this.endpoints); - return cloned; - } - - /** - * Get the list of endpoints. - * @return the endpoint list. - */ - public List getEndpoints() { - return endpoints; - } - - /** - * Add an endpoint registration - * @param endpoint endpoint (non-null) - */ - public void addEndpoint(Endpoint endpoint) { - Preconditions.checkArgument(endpoint != null); - endpoints.add(endpoint); - } - - /** - * Register a port endpoint as an inet-addr formatted endpoint, using the - * hostname as the first part of the address - * @param port port port - * @param api API API name - */ - public void registerPortEndpoint(int port, String api) { - Endpoint epr = - RegistryTypeUtils.inetAddrEndpoint(api, - ProtocolTypes.PROTOCOL_TCP, host, port); - addEndpoint(epr); - } - - /** - * Serialize. Some data structures (e.g output) - * may be shared - * @return a serialized form for marshalling as JSON - */ - public ContainerInformation serialize() { - ContainerInformation info = new ContainerInformation(); - info.containerId = id; - info.component = role; - info.appVersion = appVersion; - info.startTime = startTime; - info.createTime = createTime; - info.diagnostics = diagnostics; - info.state = state; - info.host = host; - info.hostURL = hostURL; - info.released = released ? Boolean.TRUE : null; - if (placement != null) { - info.placement = placement.toString(); - } - if (output != null) { - info.output = output; - } - return info; - } - - public String getCompInstanceName() { - return compInstanceName; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java deleted file mode 100644 index ec0ff25..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java +++ /dev/null @@ -1,367 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import com.codahale.metrics.Metric; -import com.codahale.metrics.MetricSet; -import com.google.common.base.Preconditions; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.slider.api.types.ComponentInformation; -import org.apache.slider.api.types.RoleStatistics; -import org.apache.slider.providers.PlacementPolicy; -import org.apache.slider.providers.ProviderRole; -import org.apache.slider.server.appmaster.management.BoolMetricPredicate; -import org.apache.hadoop.yarn.service.metrics.ServiceMetrics; - -import java.io.Serializable; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Map; - -/** - * Models the ongoing status of all nodes in an application. - * - * These structures are shared across the {@link AppState} and {@link RoleHistory} structures, - * and must be designed for synchronous access. Atomic counters are preferred to anything which - * requires synchronization. Where synchronized access is good is that it allows for - * the whole instance to be locked, for updating multiple entries. - */ -public final class RoleStatus implements MetricSet { - - private final String name; - - /** - * Role priority - */ - private final int key; - private final ProviderRole providerRole; - - /** resource requirements */ - private Resource resourceRequirements; - private ServiceMetrics componentMetrics; - - /** any pending AA request */ - private volatile OutstandingRequest outstandingAArequest = null; - - - private String failureMessage = ""; - - public RoleStatus(ProviderRole providerRole) { - this.providerRole = providerRole; - this.name = providerRole.name; - this.key = providerRole.id; - componentMetrics = - ServiceMetrics.register(this.name, "Metrics for component " + this.name); - componentMetrics - .tag("type", "Metrics type [component or service]", "component"); - } - - public ServiceMetrics getComponentMetrics() { - return this.componentMetrics; - } - - @Override - public Map getMetrics() { - Map metrics = new HashMap<>(15); - metrics.put("outstandingAArequest", - new BoolMetricPredicate(new BoolMetricPredicate.Eval() { - @Override - public boolean eval() { - return isAARequestOutstanding(); - } - })); - return metrics; - } - - public String getName() { - return name; - } - - public int getKey() { - return key; - } - - public int getPriority() { - return getKey(); - } - - /** - * Get the placement policy enum, from the values in - * {@link PlacementPolicy} - * @return the placement policy for this role - */ - public int getPlacementPolicy() { - return providerRole.placementPolicy; - } - - public long getPlacementTimeoutSeconds() { - return providerRole.placementTimeoutSeconds; - } - - /** - * The number of failures on a specific node that can be tolerated - * before selecting a different node for placement - * @return - */ - public int getNodeFailureThreshold() { - return providerRole.nodeFailureThreshold; - } - - public boolean isExcludeFromFlexing() { - return hasPlacementPolicy(PlacementPolicy.EXCLUDE_FROM_FLEXING); - } - - public boolean isStrictPlacement() { - return hasPlacementPolicy(PlacementPolicy.STRICT); - } - - public boolean isAntiAffinePlacement() { - return hasPlacementPolicy(PlacementPolicy.ANTI_AFFINITY_REQUIRED); - } - - public boolean hasPlacementPolicy(int policy) { - return 0 != (getPlacementPolicy() & policy); - } - - public boolean isPlacementDesired() { - return !hasPlacementPolicy(PlacementPolicy.ANYWHERE); - } - - /** - * Probe for an outstanding AA request being true - * @return true if there is an outstanding AA Request - */ - public boolean isAARequestOutstanding() { - return outstandingAArequest != null; - } - - /** - * expose the predicate {@link #isAARequestOutstanding()} as an integer, - * which is very convenient in tests - * @return 1 if there is an outstanding request; 0 if not - */ - public int getOutstandingAARequestCount() { - return isAARequestOutstanding()? 1: 0; - } - /** - * Note that a role failed, text will - * be used in any diagnostics if an exception - * is later raised. - * @param text text about the failure - */ - public synchronized void noteFailed(String text) { - if (text != null) { - failureMessage = text; - } - } - - - public void setOutstandingAArequest(OutstandingRequest outstandingAArequest) { - this.outstandingAArequest = outstandingAArequest; - } - - /** - * Complete the outstanding AA request (there's no check for one in progress, caller - * expected to have done that). - */ - public void completeOutstandingAARequest() { - setOutstandingAArequest(null); - } - - /** - * Cancel any outstanding AA request. Harmless if the role is non-AA, or - * if there are no outstanding requests. - */ - public void cancelOutstandingAARequest() { - if (outstandingAArequest != null) { - setOutstandingAArequest(null); - } - } - - public long getDesired() { - return componentMetrics.containersDesired.value(); - } - - public void setDesired(int desired) { - componentMetrics.containersDesired.set(desired); - } - - public long getRunning() { - return componentMetrics.containersRunning.value(); - } - - public long getRequested() { - return componentMetrics.containersRequested.value(); - } - - public long getAAPending() { - return componentMetrics.pendingAAContainers.value(); - } - - void decAAPending() { - componentMetrics.pendingAAContainers.decr(); - } - - void setAAPending(long n) { - componentMetrics.pendingAAContainers.set((int)n); - } - - public long getLimitsExceeded() { - return componentMetrics.containersLimitsExceeded.value(); - } - - public long getPreempted() { - return componentMetrics.containersPreempted.value(); - } - - public long getDiskFailed() { - return componentMetrics.containersDiskFailure.value(); - } - - public long getFailedRecently() { -// return componentMetrics.failedSinceLastThreshold.value(); - return 0; - } - - public long resetFailedRecently() { -// long count = -// componentMetrics.failedSinceLastThreshold.value(); -// componentMetrics.failedSinceLastThreshold.set(0); - return 0; - } - - public long getFailed() { - return componentMetrics.containersFailed.value(); - } - - String getFailureMessage() { - return this.failureMessage; - } - /** - * Get the number of roles we are short of. - * nodes released are ignored. - * @return the positive or negative number of roles to add/release. - * 0 means "do nothing". - */ - public long getDelta() { - long inuse = getActualAndRequested(); - long delta = getDesired() - inuse; - if (delta < 0) { - // TODO this doesn't do anything now that we're not tracking releasing - // containers -- maybe we need releasing - //if we are releasing, remove the number that are already released. - //but never switch to a positive - // TODO, WHY is this min operation even needed ??? if delta is negative, it's always < 0 ??? - delta = Math.min(delta, 0); - } - return delta; - } - - /** - * Get count of actual and requested containers. - * @return the size of the application when outstanding requests are included. - */ - public long getActualAndRequested() { - return getRunning() + getRequested(); - } - - /** - * Get the provider role - * @return the provider role - */ - public ProviderRole getProviderRole() { - return providerRole; - } - - /** - * Produced a serialized form which can be served up as JSON - * @return a summary of the current role status. - */ - public synchronized ComponentInformation serialize() { - ComponentInformation info = new ComponentInformation(); - info.name = name; - return info; - } - - /** - * Get the (possibly null) label expression for this role - * @return a string or null - */ - public String getLabelExpression() { - return providerRole.labelExpression; - } - - public Resource getResourceRequirements() { - return resourceRequirements; - } - - public void setResourceRequirements(Resource resourceRequirements) { - this.resourceRequirements = resourceRequirements; - } - - /** - * Compare two role status entries by name - */ - public static class CompareByName implements Comparator, - Serializable { - @Override - public int compare(RoleStatus o1, RoleStatus o2) { - return o1.getName().compareTo(o2.getName()); - } - } - - /** - * Compare two role status entries by key - */ - public static class CompareByKey implements Comparator, - Serializable { - @Override - public int compare(RoleStatus o1, RoleStatus o2) { - return (o1.getKey() < o2.getKey() ? -1 : (o1.getKey() == o2.getKey() ? 0 : 1)); - } - } - - /** - * Given a resource, set its requirements to those this role needs - * @param resource resource to configure - * @return the resource - */ - public Resource copyResourceRequirements(Resource resource) { - Preconditions.checkNotNull(resourceRequirements, - "Role resource requirements have not been set"); - resource.setMemory(resourceRequirements.getMemory()); - resource.setVirtualCores(resourceRequirements.getVirtualCores()); - return resource; - } - - public synchronized RoleStatistics getStatistics() { - RoleStatistics stats = new RoleStatistics(); - stats.activeAA = getOutstandingAARequestCount(); - stats.actual = getRunning(); - stats.desired = getDesired(); - stats.failed = getFailed(); - stats.limitsExceeded = getLimitsExceeded(); - stats.nodeFailed = getDiskFailed(); - stats.preempted = getPreempted(); - stats.requested = getRequested(); - stats.started = getRunning(); - return stats; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/SimpleReleaseSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/SimpleReleaseSelector.java deleted file mode 100644 index b848096..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/SimpleReleaseSelector.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import java.util.List; - -/** - * Simplest release selector simply returns the list - */ -public class SimpleReleaseSelector implements ContainerReleaseSelector { - - @Override - public List sortCandidates(int roleId, - List candidates) { - return candidates; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java deleted file mode 100644 index 90221cb..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java +++ /dev/null @@ -1,275 +0,0 @@ -/* - * 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.slider.server.appmaster.state; - -import com.google.common.cache.LoadingCache; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import org.apache.slider.api.ClusterNode; -import org.apache.slider.api.StatusKeys; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.ConfigFile; -import org.apache.slider.api.types.ApplicationLivenessInformation; -import org.apache.slider.api.types.ComponentInformation; -import org.apache.slider.api.types.NodeInformation; -import org.apache.slider.api.types.RoleStatistics; -import org.apache.slider.core.exceptions.NoSuchNodeException; -import org.apache.slider.core.registry.docstore.PublishedConfigSet; -import org.apache.slider.core.registry.docstore.PublishedExportsSet; - -import java.util.Collection; -import java.util.List; -import java.util.Map; - -/** - * The methods to offer state access to the providers and other parts of - * the system which want read-only access to the state. - */ -public interface StateAccessForProviders { - - /** - * Get a map of role status entries by role Id - * @return the map of currently defined roles. - */ - Map getRoleStatusMap(); - - /** - * Get the name of the application - * @return the name - */ - String getApplicationName(); - - /** - * Get the published configurations - * @return the configuration set - */ - PublishedConfigSet getPublishedSliderConfigurations(); - - /** - * Get the published exports set - * @return - */ - PublishedExportsSet getPublishedExportsSet(); - - /** - * Get a named published config set - * @param name name to look up - * @return the instance or null - */ - PublishedConfigSet getPublishedConfigSet(String name); - - /** - * Get a named published config set, creating it if need be. - * @param name name to look up - * @return the instance -possibly a new one - */ - PublishedConfigSet getOrCreatePublishedConfigSet(String name); - - /** - * List the config sets -this takes a clone of the current set - * @return a list of config sets - */ - List listConfigSets(); - - /** - * Get a map of all the failed containers - * @return map of recorded failed containers - */ - Map getFailedContainers(); - - /** - * Get the live containers. - * - * @return the live nodes - */ - Map getLiveContainers(); - - /** - * Get the current cluster description - * @return the actual state of the cluster - */ - Application getApplication(); - - /** - * Flag set to indicate the application is live -this only happens - * after the buildInstance operation - */ - boolean isApplicationLive(); - - /** - * Look up a role from its key -or fail - * - * @param key key to resolve - * @return the status - * @throws YarnRuntimeException on no match - */ - RoleStatus lookupRoleStatus(int key); - - /** - * Look up a role from its key -or fail - * - * @param c container in a role - * @return the status - * @throws YarnRuntimeException on no match - */ - RoleStatus lookupRoleStatus(Container c) throws YarnRuntimeException; - - /** - * Look up a role from its key -or fail - * - * @param name container in a role - * @return the status - * @throws YarnRuntimeException on no match - */ - RoleStatus lookupRoleStatus(String name) throws YarnRuntimeException; - - /** - * Clone a list of active containers - * @return the active containers at the time - * the call was made - */ - List cloneOwnedContainerList(); - - /** - * Get the number of active containers - * @return the number of active containers the time the call was made - */ - int getNumOwnedContainers(); - - /** - * Get any active container with the given ID - * @param id container Id - * @return the active container or null if it is not found - */ - RoleInstance getOwnedContainer(ContainerId id); - - /** - * Get any active container with the given ID - * @param id container Id - * @return the active container or null if it is not found - */ - RoleInstance getOwnedContainer(String id) throws NoSuchNodeException; - - /** - * Create a clone of the list of live cluster nodes. - * @return the list of nodes, may be empty - */ - List cloneLiveContainerInfoList(); - - /** - * Get the {@link RoleInstance} details on a container. - * This is an O(n) operation - * @param containerId the container ID - * @return null if there is no such node - * @throws NoSuchNodeException if the node cannot be found - */ - RoleInstance getLiveInstanceByContainerID(String containerId) - throws NoSuchNodeException; - - /** - * Get the details on a list of instaces referred to by ID. - * Unknown nodes are not returned - * Important: the order of the results are undefined - * @param containerIDs the containers - * @return list of instances - */ - List getLiveInstancesByContainerIDs( - Collection containerIDs); - - /** - * Update the cluster description with anything interesting - */ - Application refreshClusterStatus(); - - - /** - * get application liveness information - * @return a snapshot of the current liveness information - */ - ApplicationLivenessInformation getApplicationLivenessInformation(); - - /** - * Get a snapshot of component information. - *

- * This does not include any container list, which - * is more expensive to create. - * @return a map of current role status values. - */ - Map getComponentInfoSnapshot(); - - /** - * Find out about the nodes for specific roles - * Component_name -> ContainerId -> ClusterNode - * @return - */ - Map> getRoleClusterNodeMapping(); - - /** - * Enum all role instances by role. - * @param role role, or "" for all roles - * @return a list of instances, may be empty - */ - List enumLiveInstancesInRole(String role); - - /** - * Look up all containers of a specific component name - * @param component component/role name - * @return list of instances. This is a snapshot - */ - List lookupRoleContainers(String component); - - /** - * Get the JSON serializable information about a component - * @param component component to look up - * @return a structure describing the component. - */ - ComponentInformation getComponentInformation(String component); - - - /** - * Get a clone of the nodemap. - * The instances inside are not cloned - * @return a possibly empty map of hostname top info - */ - Map getNodeInformationSnapshot(); - - /** - * get information on a node - * @param hostname hostname to look up - * @return the information, or null if there is no information held. - */ - NodeInformation getNodeInformation(String hostname); - - /** - * Get the aggregate statistics across all roles - * @return role statistics - */ - RoleStatistics getRoleStatistics(); - - /** - * Get global substitution tokens. - */ - Map getGlobalSubstitutionTokens(); - - /** - * Get config file cache. - */ - LoadingCache getConfigFileCache(); -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/HttpCacheHeaders.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/HttpCacheHeaders.java deleted file mode 100644 index be8960d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/HttpCacheHeaders.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.slider.server.appmaster.web; - -/* - - - , ); - long now = System.currentTimeMillis(); - httpRes.addDateHeader ( "Expires", now ); - httpRes.addDateHeader ( "Date", now ); - httpRes.addHeader ( "Pragma", "no-cache" ); - */ -public interface HttpCacheHeaders { - String HTTP_HEADER_CACHE_CONTROL = "Cache-Control"; - String HTTP_HEADER_CACHE_CONTROL_NONE = "no-cache"; - String HTTP_HEADER_PRAGMA = "Pragma"; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMController.java deleted file mode 100644 index c3c6e60..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMController.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.slider.server.appmaster.web; - -import com.google.inject.Inject; -import org.apache.hadoop.yarn.webapp.Controller; -import org.apache.slider.server.appmaster.web.layout.AppLayout; -import org.apache.slider.server.appmaster.web.layout.ClusterSpecificationView; -import org.apache.slider.server.appmaster.web.layout.ContainerStatsView; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * - */ -public class SliderAMController extends Controller { - private static final Logger log = LoggerFactory.getLogger(SliderAMController.class); - - private final WebAppApi slider; - - @Inject - public SliderAMController(WebAppApi slider, RequestContext ctx) { - super(ctx); - this.slider = slider; - } - - @Override - public void index() { - setTitle("Slider App Master"); - - updateAppState(); - - render(AppLayout.class); - } - - public void containerStats() { - setTitle("Container Statistics"); - - updateAppState(); - - render(ContainerStatsView.class); - } - - public void specification() { - setTitle("Cluster Specification"); - - render(ClusterSpecificationView.class); - } - - private void updateAppState() { - //TODO don't do this on every request? - slider.getAppState().refreshClusterStatus(); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java deleted file mode 100644 index 0cac430..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * 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.slider.server.appmaster.web; - -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.health.HealthCheckRegistry; -import com.codahale.metrics.servlets.HealthCheckServlet; -import com.codahale.metrics.servlets.MetricsServlet; -import com.codahale.metrics.servlets.PingServlet; -import com.codahale.metrics.servlets.ThreadDumpServlet; -import com.google.common.base.Preconditions; -import com.sun.jersey.api.container.filter.GZIPContentEncodingFilter; -import com.sun.jersey.api.core.ResourceConfig; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import com.sun.jersey.spi.container.servlet.ServletContainer; -import org.apache.hadoop.yarn.webapp.Dispatcher; -import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; -import org.apache.hadoop.yarn.webapp.WebApp; -import org.apache.slider.server.appmaster.web.rest.AMWadlGeneratorConfig; -import org.apache.slider.server.appmaster.web.rest.AMWebServices; -import org.apache.slider.server.appmaster.web.rest.SliderJacksonJaxbJsonProvider; - -import java.util.HashMap; -import java.util.Map; -import java.util.logging.Level; -import java.util.logging.Logger; - -import static org.apache.slider.server.appmaster.web.rest.RestPaths.*; - -/** - * - */ -public class SliderAMWebApp extends WebApp { - public static final String BASE_PATH = "slideram"; - public static final String CONTAINER_STATS = "/stats"; - public static final String CLUSTER_SPEC = "/spec"; - - private final WebAppApi webAppApi; - - public SliderAMWebApp(WebAppApi webAppApi) { - Preconditions.checkArgument(webAppApi != null, "webAppApi null"); - this.webAppApi = webAppApi; - } - - @Override - public void setup() { - Logger.getLogger("com.sun.jersey").setLevel(Level.FINEST); - // Make one of these to ensure that the jax-b annotations - // are properly picked up. - bind(SliderJacksonJaxbJsonProvider.class); - - // Get exceptions printed to the screen - bind(GenericExceptionHandler.class); - // bind the REST interface - bind(AMWebServices.class); - - //bind(AMAgentWebServices.class); - route("/", SliderAMController.class); - route(CONTAINER_STATS, SliderAMController.class, "containerStats"); - route(CLUSTER_SPEC, SliderAMController.class, "specification"); - } - - @Override - public void configureServlets() { - setup(); - - serve("/", "/__stop").with(Dispatcher.class); - - for (String path : this.getServePathSpecs()) { - serve(path).with(Dispatcher.class); - } - - serve(SYSTEM_HEALTHCHECK) - .with(new HealthCheckServlet(new HealthCheckRegistry())); - serve(SYSTEM_METRICS).with(new MetricsServlet(new MetricRegistry())); - serve(SYSTEM_PING).with(new PingServlet()); - serve(SYSTEM_THREADS).with(new ThreadDumpServlet()); - - String regex = "(?!/ws)"; - serveRegex(regex).with(SliderDefaultWrapperServlet.class); - - Map params = new HashMap<>(); - params.put(ResourceConfig.FEATURE_IMPLICIT_VIEWABLES, "true"); - params.put(ServletContainer.FEATURE_FILTER_FORWARD_ON_404, "true"); - params.put(ResourceConfig.FEATURE_XMLROOTELEMENT_PROCESSING, "true"); - params.put(ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS, GZIPContentEncodingFilter.class.getName()); - params.put(ResourceConfig.PROPERTY_CONTAINER_RESPONSE_FILTERS, GZIPContentEncodingFilter.class.getName()); - //params.put("com.sun.jersey.spi.container.ContainerRequestFilters", "com.sun.jersey.api.container.filter.LoggingFilter"); - //params.put("com.sun.jersey.spi.container.ContainerResponseFilters", "com.sun.jersey.api.container.filter.LoggingFilter"); - //params.put("com.sun.jersey.config.feature.Trace", "true"); - params.put("com.sun.jersey.config.property.WadlGeneratorConfig", - AMWadlGeneratorConfig.CLASSNAME); - filter("/*").through(GuiceContainer.class, params); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderDefaultWrapperServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderDefaultWrapperServlet.java deleted file mode 100644 index 12c41ac..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderDefaultWrapperServlet.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.slider.server.appmaster.web; - -import com.google.inject.Singleton; -import org.apache.hadoop.yarn.webapp.DefaultWrapperServlet; - -import javax.servlet.RequestDispatcher; -import javax.servlet.ServletException; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletRequestWrapper; -import javax.servlet.http.HttpServletResponse; -import java.io.IOException; - -/** - * - */ -@Singleton -public class SliderDefaultWrapperServlet extends DefaultWrapperServlet { - @Override - protected void doPost(HttpServletRequest req, HttpServletResponse resp) - throws ServletException, IOException { - RequestDispatcher rd = getServletContext().getNamedDispatcher("default"); - - HttpServletRequest wrapped = new HttpServletRequestWrapper(req) { - public String getServletPath() { - return ""; - } - }; - - rd.forward(wrapped, resp); - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java deleted file mode 100644 index 02f3f0c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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.slider.server.appmaster.web; - -import org.apache.hadoop.registry.client.api.RegistryOperations; -import org.apache.slider.server.appmaster.actions.QueueAccess; -import org.apache.slider.server.appmaster.management.MetricsAndMonitoring; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.StateAccessForProviders; - -/** - * Interface to pass information from the Slider AppMaster to the WebApp - */ -public interface WebAppApi { - - /** - * The {@link AppState} for the current cluster - */ - StateAccessForProviders getAppState(); - - /** - * Registry operations accessor - * @return registry access - */ - RegistryOperations getRegistryOperations(); - - /** - * Metrics and monitoring service - * @return the (singleton) instance - */ - MetricsAndMonitoring getMetricsAndMonitoring(); - - /** - * Get the queue accessor - * @return the immediate and scheduled queues - */ - QueueAccess getQueues(); -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java deleted file mode 100644 index f88f501..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.slider.server.appmaster.web; - -import org.apache.hadoop.registry.client.api.RegistryOperations; -import org.apache.slider.server.appmaster.actions.QueueAccess; -import org.apache.slider.server.appmaster.management.MetricsAndMonitoring; -import org.apache.slider.server.appmaster.state.StateAccessForProviders; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static com.google.common.base.Preconditions.checkNotNull; - -/** - * - */ -public class WebAppApiImpl implements WebAppApi { - private static final Logger log = LoggerFactory.getLogger(WebAppApiImpl.class); - - protected final StateAccessForProviders appState; - private final RegistryOperations registryOperations; - private final MetricsAndMonitoring metricsAndMonitoring; - private final QueueAccess queues; - - public WebAppApiImpl(StateAccessForProviders appState, - RegistryOperations registryOperations, - MetricsAndMonitoring metricsAndMonitoring, QueueAccess queues) { - checkNotNull(appState); - this.queues = queues; - - this.registryOperations = registryOperations; - this.appState = appState; - this.metricsAndMonitoring = metricsAndMonitoring; - } - - @Override - public StateAccessForProviders getAppState() { - return appState; - } - - @Override - public RegistryOperations getRegistryOperations() { - return registryOperations; - } - - @Override - public MetricsAndMonitoring getMetricsAndMonitoring() { - return metricsAndMonitoring; - } - - @Override - public QueueAccess getQueues() { - return queues; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/AppLayout.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/AppLayout.java deleted file mode 100644 index d9a2cda..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/AppLayout.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.slider.server.appmaster.web.layout; - -import org.apache.hadoop.yarn.webapp.SubView; -import org.apache.slider.server.appmaster.web.view.IndexBlock; - -/** - * - */ -public class AppLayout extends WebUILayout { - - @Override - protected Class content() { - return IndexBlock.class; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ClusterSpecificationView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ClusterSpecificationView.java deleted file mode 100644 index b54ca71..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ClusterSpecificationView.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.slider.server.appmaster.web.layout; - -import org.apache.hadoop.yarn.webapp.SubView; -import org.apache.slider.server.appmaster.web.view.ClusterSpecificationBlock; - -/** - * - */ -public class ClusterSpecificationView extends WebUILayout { - - @Override - protected Class content() { - return ClusterSpecificationBlock.class; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ContainerStatsView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ContainerStatsView.java deleted file mode 100644 index 39ba0ad..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ContainerStatsView.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.slider.server.appmaster.web.layout; - -import org.apache.hadoop.yarn.webapp.SubView; -import org.apache.slider.server.appmaster.web.view.ContainerStatsBlock; - - - -/** - * - */ -public class ContainerStatsView extends WebUILayout { - - @Override - protected Class content() { - return ContainerStatsBlock.class; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/WebUILayout.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/WebUILayout.java deleted file mode 100644 index 1681f59..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/WebUILayout.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.slider.server.appmaster.web.layout; - -import org.apache.hadoop.yarn.webapp.SubView; -import org.apache.hadoop.yarn.webapp.view.TwoColumnLayout; -import org.apache.slider.server.appmaster.web.view.NavBlock; - -import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION; -import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION_ID; -import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID; - -/** - * - */ -public class WebUILayout extends TwoColumnLayout { - - @Override - protected void preHead(Page.HTML<_> html) { - set(ACCORDION_ID, "nav"); - set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}"); - } - - @Override - protected Class nav() { - return NavBlock.class; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGenerator.java deleted file mode 100644 index 05aaa5b..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGenerator.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest; - -import com.sun.jersey.server.wadl.ApplicationDescription; -import com.sun.jersey.server.wadl.WadlGenerator; -import com.sun.jersey.server.wadl.WadlGeneratorImpl; -import com.sun.research.ws.wadl.Application; -import com.sun.research.ws.wadl.Resource; -import com.sun.research.ws.wadl.Resources; - -import java.util.Iterator; -import java.util.List; - -/** - * - */ -public class AMWadlGenerator extends WadlGeneratorImpl { - @Override - /** - * This method is called once the WADL application has been assembled, so it - * affords an opportunity to edit the resources presented by the WADL. In - * this case, we're removing the internal "/agents" resources. - */ - public void attachTypes(ApplicationDescription egd) { - super.attachTypes(egd); - - Application application = egd.getApplication(); - List resources = application.getResources(); - - for (Resources appResources : resources) { - List resourceList = appResources.getResource(); - for (Resource appResource : resourceList) { - String path = appResource.getPath(); - if (RestPaths.SLIDER_CONTEXT_ROOT.equals(path)) { - List sliderResources = appResource.getMethodOrResource(); - Iterator itor = sliderResources.iterator(); - while (itor.hasNext()) { - Object sliderRes = itor.next(); - if (sliderRes instanceof Resource) { - Resource res = (Resource) sliderRes; - if (RestPaths.SLIDER_SUBPATH_AGENTS.equals(res.getPath())) { - // assuming I'll get a list modification issue if I remove at this - // point - itor.remove(); - } - } - } - } - } - } - } - - @Override - public void setWadlGeneratorDelegate(WadlGenerator delegate) { - // do nothing - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGeneratorConfig.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGeneratorConfig.java deleted file mode 100644 index ea9f22b..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGeneratorConfig.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest; - -import com.sun.jersey.api.wadl.config.WadlGeneratorConfig; -import com.sun.jersey.api.wadl.config.WadlGeneratorDescription; - -import java.util.List; - -/** - * App master's WADL generation support - */ -public class AMWadlGeneratorConfig extends WadlGeneratorConfig { - - public static final String CLASSNAME = "org.apache.slider.server.appmaster.web.rest.AMWadlGeneratorConfig"; - @Override - public List configure() { - return generator(AMWadlGenerator.class).descriptions(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java deleted file mode 100644 index 44259d3..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest; - -import com.google.inject.Inject; -import com.google.inject.Singleton; -import org.apache.slider.api.resource.Application; -import org.apache.slider.server.appmaster.web.WebAppApi; -import org.apache.slider.server.appmaster.web.rest.application.actions.RestActionStop; -import org.apache.slider.server.appmaster.web.rest.application.actions.StopResponse; -import org.apache.slider.server.appmaster.web.rest.publisher.PublisherResource; -import org.apache.slider.server.appmaster.web.rest.registry.RegistryResource; - -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.GET; -import javax.ws.rs.POST; -import javax.ws.rs.Path; -import javax.ws.rs.Produces; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.UriInfo; - -import static javax.ws.rs.core.MediaType.APPLICATION_JSON; -import static org.apache.slider.server.appmaster.web.rest.RestPaths.ACTION_STOP; - -/** - * The available REST services exposed by a slider AM. - */ -@Singleton -@Path(RestPaths.SLIDER_CONTEXT_ROOT) -public class AMWebServices { - - /** AM/WebApp info object */ - private WebAppApi slider; - private final PublisherResource publisherResource; - private final RegistryResource registryResource; - - @Inject - public AMWebServices(WebAppApi slider) { - this.slider = slider; - publisherResource = new PublisherResource(slider); - registryResource = new RegistryResource(slider); - } - //TODO add an endpoint for exposing configs - - @Path(RestPaths.SLIDER_SUBPATH_PUBLISHER) - public PublisherResource getPublisherResource() { - return publisherResource; - } - - @Path(RestPaths.SLIDER_SUBPATH_REGISTRY) - public RegistryResource getRegistryResource() { - return registryResource; - } - - - @GET - @Path(RestPaths.SLIDER_SUBPATH_APPLICATION) - @Produces({APPLICATION_JSON}) - public Application getApplicationResource() { - return slider.getAppState().getApplication(); - } - - @POST - @Path(ACTION_STOP) - @Produces({APPLICATION_JSON}) - public StopResponse actionStop(@Context HttpServletRequest request, - @Context UriInfo uriInfo, - String body) { - return new RestActionStop(slider).stop(request, uriInfo, body); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java deleted file mode 100644 index 7ff83b6..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest; - -import org.apache.hadoop.fs.PathNotFoundException; -import org.apache.hadoop.registry.client.exceptions.AuthenticationFailedException; -import org.apache.hadoop.registry.client.exceptions.NoPathPermissionsException; -import org.apache.hadoop.yarn.webapp.ForbiddenException; -import org.apache.hadoop.yarn.webapp.NotFoundException; -import org.apache.slider.server.appmaster.management.MetricsAndMonitoring; -import org.apache.slider.server.appmaster.web.WebAppApi; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.Response; -import java.io.FileNotFoundException; -import java.net.URI; -import java.net.URL; - -/** - * Abstract resource base class for REST resources - * that use the slider WebAppApi - */ -public abstract class AbstractSliderResource { - private static final Logger log = - LoggerFactory.getLogger(AbstractSliderResource.class); - protected final WebAppApi slider; - protected final MetricsAndMonitoring metricsAndMonitoring; - - protected AbstractSliderResource(WebAppApi slider) { - this.slider = slider; - metricsAndMonitoring = slider.getMetricsAndMonitoring(); - } - - /** - * Generate a redirect to the WASL - * @param request to base the URL on - * @return a 302 response - */ - protected Response redirectToAppWadl(HttpServletRequest request) { - try { - URI location = new URL(request.getScheme(), - request.getServerName(), - request.getServerPort(), - RestPaths.APPLICATION_WADL).toURI(); - return Response.temporaryRedirect(location).build(); - } catch (Exception e) { - log.error("Error during redirect to WADL", e); - throw new WebApplicationException(Response.serverError().build()); - } - } - - /** - * Convert any exception caught into a web application - * exception for rethrowing - * @param path path of request - * @param ex exception - * @return an exception to throw - */ - public WebApplicationException buildException(String path, - Exception ex) { - try { - throw ex; - } catch (WebApplicationException e) { - // rethrow direct - throw e; - } catch (FileNotFoundException e) { - return new NotFoundException("Not found: " + path); - } catch (PathNotFoundException e) { - return new NotFoundException("Not found: " + path); - } catch (AuthenticationFailedException e) { - return new ForbiddenException(path); - } catch (NoPathPermissionsException e) { - return new ForbiddenException(path); - } catch (Exception e) { - log.error("Error during generation of response: {}", e, e); - return new WebApplicationException(e); - } - } - - /** - * Mark an GET operation on a path - * @param verb HTTP Verb - * @param path path relative to slider API - */ - protected void mark(String verb, String path) { - metricsAndMonitoring.markMeterAndCounter(verb + "-" + path); - } - - /** - * Mark an GET operation on a path - * @param verb HTTP Verb - * @param path path relative to slider API - */ - protected void mark(String verb, String path, String subpath) { - metricsAndMonitoring.markMeterAndCounter(verb + "-" + path + subpath); - } - - /** - * Mark a GET operation on a path - * @param path path relative to slider API - */ - protected void markGet(String path) { - mark("GET", path); - } - - /** - * Mark a GET operation on a path - * @param path path relative to slider API - */ - protected void markGet(String path, String subpath) { - mark("GET", path, subpath); - } - - /** - * Mark a GET operation on a path - * @param path path relative to slider API - */ - protected void markPost(String path, String subpath) { - mark("POST", path, subpath); - } - - /** - * Mark a GET operation on a path - * @param path path relative to slider API - */ - protected void markPut(String path, String subpath) { - mark("PUT", path, subpath); - } - - /** - * Mark a GET operation on a path - * @param path path relative to slider API - */ - protected void markDelete(String path, String subpath) { - mark("DELETE", path, subpath); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java deleted file mode 100644 index b4a92ba..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest; - -import org.apache.hadoop.yarn.server.webproxy.WebAppProxyServlet; -import org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter; -import org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpPrincipal; -import org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpServletRequestWrapper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.servlet.FilterChain; -import javax.servlet.FilterConfig; -import javax.servlet.ServletException; -import javax.servlet.ServletRequest; -import javax.servlet.ServletResponse; -import javax.servlet.http.Cookie; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; -import java.io.IOException; - -/** - * This is a filter which is used to forward insecure operations - * There's some metrics to track all operations too - */ -public class InsecureAmFilter extends AmIpFilter { - public static final String WS_CONTEXT_ROOT = "slider.rest.context.root"; - protected static final Logger log = - LoggerFactory.getLogger(InsecureAmFilter.class); - - private String wsContextRoot; - - - @Override - public void init(FilterConfig conf) throws ServletException { - super.init(conf); - wsContextRoot = conf.getInitParameter(WS_CONTEXT_ROOT); - if (wsContextRoot == null) { - throw new ServletException("No value set for " + WS_CONTEXT_ROOT); - } - } - - private void rejectNonHttpRequests(ServletRequest req) throws - ServletException { - if (!(req instanceof HttpServletRequest)) { - throw new ServletException("This filter only works for HTTP/HTTPS"); - } - } - - @Override - public void doFilter(ServletRequest req, - ServletResponse resp, - FilterChain chain) throws IOException, ServletException { - rejectNonHttpRequests(req); - HttpServletRequest httpReq = (HttpServletRequest) req; - - - String requestURI = httpReq.getRequestURI(); - if (requestURI == null || !requestURI.startsWith(wsContextRoot)) { - // hand off to the AM filter if it is not the context root - super.doFilter(req, resp, chain); - return; - } - - String user = null; - - if (httpReq.getCookies() != null) { - for (Cookie c : httpReq.getCookies()) { - if (WebAppProxyServlet.PROXY_USER_COOKIE_NAME.equals(c.getName())) { - user = c.getValue(); - break; - } - } - } - - if (user == null) { - log.debug("Could not find " + WebAppProxyServlet.PROXY_USER_COOKIE_NAME - + " cookie, so user will not be set"); - chain.doFilter(req, resp); - } else { - final AmIpPrincipal principal = new AmIpPrincipal(user); - ServletRequest requestWrapper = new AmIpServletRequestWrapper(httpReq, - principal); - chain.doFilter(requestWrapper, resp); - } - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java deleted file mode 100644 index 42a5bdd..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java +++ /dev/null @@ -1,102 +0,0 @@ -/** - * 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.slider.server.appmaster.web.rest; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.http.FilterContainer; -import org.apache.hadoop.http.FilterInitializer; -import org.apache.hadoop.http.HttpConfig; -import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.webapp.util.WebAppUtils; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class InsecureAmFilterInitializer extends FilterInitializer { - private static final String FILTER_NAME = "AM_PROXY_FILTER"; - private static final String FILTER_CLASS = - InsecureAmFilter.class.getCanonicalName(); - private static final String HTTPS_PREFIX = "https://"; - private static final String HTTP_PREFIX = "http://"; - - static final String PROXY_HOSTS = "PROXY_HOSTS"; - static final String PROXY_HOSTS_DELIMITER = ","; - static final String PROXY_URI_BASES = "PROXY_URI_BASES"; - static final String PROXY_URI_BASES_DELIMITER = ","; - - private Configuration configuration; - - public static final String NAME = - "org.apache.slider.server.appmaster.web.rest.InsecureAmFilterInitializer"; - - @Override - public void initFilter(FilterContainer container, Configuration conf) { - configuration = conf; - Map params = new HashMap(); - String proxy = WebAppUtils.getProxyHostAndPort(conf); - String[] parts = proxy.split(":"); - params.put(InsecureAmFilter.PROXY_HOST, parts[0]); - // todo: eventually call WebAppUtils.getHttpSchemePrefix - params.put(InsecureAmFilter.PROXY_URI_BASE, getHttpSchemePrefix() - + proxy + - getApplicationWebProxyBase()); - params.put(InsecureAmFilter.WS_CONTEXT_ROOT, RestPaths.WS_CONTEXT_ROOT); - container.addFilter(FILTER_NAME, FILTER_CLASS, params); - } - - private void classicAmFilterInitializerInit(FilterContainer container, - Configuration conf) { - Map params = new HashMap(); - List proxies = WebAppUtils.getProxyHostsAndPortsForAmFilter(conf); - StringBuilder sb = new StringBuilder(); - for (String proxy : proxies) { - sb.append(proxy.split(":")[0]).append(PROXY_HOSTS_DELIMITER); - } - sb.setLength(sb.length() - 1); - params.put(PROXY_HOSTS, sb.toString()); - - String prefix = WebAppUtils.getHttpSchemePrefix(conf); - String proxyBase = getApplicationWebProxyBase(); - sb = new StringBuilder(); - for (String proxy : proxies) { - sb.append(prefix).append(proxy).append(proxyBase) - .append(PROXY_HOSTS_DELIMITER); - } - sb.setLength(sb.length() - 1); - params.put(PROXY_URI_BASES, sb.toString()); - - } - - @VisibleForTesting - protected String getApplicationWebProxyBase() { - return System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV); - } - - private String getHttpSchemePrefix() { - return HttpConfig.Policy.HTTPS_ONLY == - HttpConfig.Policy.fromString(configuration - .get( - YarnConfiguration.YARN_HTTP_POLICY_KEY, - YarnConfiguration.YARN_HTTP_POLICY_DEFAULT)) - ? HTTPS_PREFIX : HTTP_PREFIX; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java deleted file mode 100644 index 581f5b4..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java +++ /dev/null @@ -1,155 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest; - -/** - * Paths in the REST App - */ -public class RestPaths { - - public static final String WS_CONTEXT = "ws"; - public static final String AGENT_WS_CONTEXT = "ws"; - - /** - * Root path for the web services context: {@value} - */ - public static final String WS_CONTEXT_ROOT = "/" + WS_CONTEXT; - - /** - * agent content root: {@value} - */ - public static final String SLIDER_CONTEXT_ROOT = WS_CONTEXT_ROOT + "/v1"; - public static final String RELATIVE_API = WS_CONTEXT + "/v1"; - public static final String MANAGEMENT = "mgmt"; - public static final String SLIDER_SUBPATH_MANAGEMENT = "/" + MANAGEMENT; - public static final String SLIDER_SUBPATH_AGENTS = "/agents"; - public static final String SLIDER_SUBPATH_PUBLISHER = "/publisher"; - - - /** - * Publisher: {@value} - */ - public static final String SLIDER_PATH_PUBLISHER = SLIDER_CONTEXT_ROOT - + SLIDER_SUBPATH_PUBLISHER; - - public static final String RELATIVE_PATH_PUBLISHER = RELATIVE_API - + SLIDER_SUBPATH_PUBLISHER; - - /** - * Registry subpath: {@value} - */ - public static final String SLIDER_SUBPATH_REGISTRY = "/registry"; - - /** - * Registry: {@value} - */ - public static final String SLIDER_PATH_REGISTRY = SLIDER_CONTEXT_ROOT - + SLIDER_SUBPATH_REGISTRY; - public static final String RELATIVE_PATH_REGISTRY = RELATIVE_API - + SLIDER_SUBPATH_REGISTRY; - - - /** - * The regular expressions used to define valid configuration names/url path - * fragments: {@value} - */ - public static final String PUBLISHED_CONFIGURATION_REGEXP - = "[a-z0-9][a-z0-9_\\+-]*"; - - public static final String PUBLISHED_CONFIGURATION_SET_REGEXP - = "[a-z0-9][a-z0-9_.\\+-]*"; - - public static final String SLIDER_CONFIGSET = "slider"; - public static final String SLIDER_EXPORTS = "exports"; - - public static final String SLIDER_CLASSPATH = "classpath"; - - /** - * Codahale Metrics - base path: {@value} - */ - - public static final String SYSTEM = "/system"; - - - - /** - * Codahale Metrics - health: {@value} - */ - public static final String SYSTEM_HEALTHCHECK = SYSTEM + "/health"; - /** - * Codahale Metrics - metrics: {@value} - */ - public static final String SYSTEM_METRICS = SYSTEM + "/metrics"; - /** - * Codahale Metrics - metrics as JSON: {@value} - */ - public static final String SYSTEM_METRICS_JSON = SYSTEM_METRICS + "?format=json"; - /** - * Codahale Metrics - ping: {@value} - */ - public static final String SYSTEM_PING = SYSTEM + "/ping"; - /** - * Codahale Metrics - thread dump: {@value} - */ - public static final String SYSTEM_THREADS = SYSTEM + "/threads"; - - /** - * application subpath - */ - public static final String SLIDER_SUBPATH_APPLICATION = "/application"; - - /** - * management path: {@value} - */ - public static final String SLIDER_PATH_APPLICATION = - SLIDER_CONTEXT_ROOT + SLIDER_SUBPATH_APPLICATION; - - - public static final String APPLICATION_WADL = "/application.wadl"; - public static final String LIVE = "/live"; - public static final String LIVE_RESOURCES = "/live/resources"; - public static final String LIVE_CONTAINERS = "/live/containers"; - public static final String LIVE_COMPONENTS = "/live/components"; - public static final String LIVE_NODES = "/live/nodes"; - public static final String LIVE_LIVENESS = "/live/liveness"; - public static final String LIVE_STATISTICS = "/live/statistics"; - public static final String MODEL = "/model"; - public static final String MODEL_DESIRED = MODEL +"/desired"; - public static final String MODEL_DESIRED_APPCONF = MODEL_DESIRED +"/appconf"; - public static final String MODEL_DESIRED_RESOURCES = MODEL_DESIRED +"/resources"; - public static final String MODEL_RESOLVED = "/model/resolved"; - public static final String MODEL_RESOLVED_APPCONF = MODEL_RESOLVED +"/appconf"; - public static final String MODEL_RESOLVED_RESOURCES = MODEL_RESOLVED +"/resources"; - public static final String MODEL_INTERNAL = "/model/internal"; - - public static final String ACTION = "/action"; - public static final String ACTION_PING = ACTION + "/ping"; - public static final String ACTION_STOP = ACTION + "/stop"; - - /** - * Path to a role - * @param name role name - * @return a path to it - */ - public String pathToRole(String name) { - - // ws/v1/slider/application/live/components/$name - return SLIDER_PATH_APPLICATION + LIVE_COMPONENTS + "/" + name; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/SliderJacksonJaxbJsonProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/SliderJacksonJaxbJsonProvider.java deleted file mode 100644 index 86d68a8..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/SliderJacksonJaxbJsonProvider.java +++ /dev/null @@ -1,58 +0,0 @@ -/** - * 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.slider.server.appmaster.web.rest; - -import com.google.inject.Singleton; -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.codehaus.jackson.jaxrs.JacksonJaxbJsonProvider; -import org.codehaus.jackson.map.AnnotationIntrospector; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.introspect.JacksonAnnotationIntrospector; -import org.codehaus.jackson.xc.JaxbAnnotationIntrospector; - -import javax.ws.rs.core.MediaType; -import javax.ws.rs.ext.Provider; - -/** - * Implementation of JAX-RS abstractions based on {@link - * JacksonJaxbJsonProvider} needed to deserialize JSON content to, or serialize - * it from, POJO objects. - */ -@Singleton -@Provider -@Unstable -@Private -public class SliderJacksonJaxbJsonProvider extends JacksonJaxbJsonProvider { - - public SliderJacksonJaxbJsonProvider() { - } - - @Override - public ObjectMapper locateMapper(Class type, MediaType mediaType) { - ObjectMapper mapper = super.locateMapper(type, mediaType); - AnnotationIntrospector introspector = new AnnotationIntrospector.Pair( - new JaxbAnnotationIntrospector(), - new JacksonAnnotationIntrospector() - ); - mapper.setAnnotationIntrospector(introspector); - //mapper.setSerializationInclusion(Inclusion.NON_NULL); - return mapper; - } -} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResouceContentCacheFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResouceContentCacheFactory.java deleted file mode 100644 index d23fcee..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResouceContentCacheFactory.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest.application; - -import org.apache.slider.server.appmaster.state.StateAccessForProviders; -import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache; - -public class ApplicationResouceContentCacheFactory { - public static final int LIFESPAN = 500; - - /** - * Build the content cache - * @param state state view - */ - public static ContentCache createContentCache( - StateAccessForProviders state) { - ContentCache cache = new ContentCache(); - return cache; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java deleted file mode 100644 index 96b4f75..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest.application.actions; - -import org.apache.slider.api.types.PingInformation; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.core.UriInfo; -import java.util.Locale; - -public class RestActionPing { - private static final Logger log = - LoggerFactory.getLogger(RestActionPing.class); - - public RestActionPing() { - } - - public PingInformation ping(HttpServletRequest request, UriInfo uriInfo, String body) { - String verb = request.getMethod(); - log.info("Ping {}", verb); - PingInformation pingInformation = new PingInformation(); - pingInformation.time = System.currentTimeMillis(); - pingInformation.verb = verb; - pingInformation.body = body; - String text = - String.format(Locale.ENGLISH, - "Ping verb %s received at %tc", - verb, pingInformation.time); - pingInformation.text = text; - return pingInformation; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java deleted file mode 100644 index 544f589..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest.application.actions; - -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.slider.core.main.LauncherExitCodes; -import org.apache.slider.server.appmaster.actions.ActionStopSlider; -import org.apache.slider.server.appmaster.web.WebAppApi; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.core.UriInfo; -import java.util.Locale; -import java.util.concurrent.TimeUnit; - -public class RestActionStop { - private static final Logger log = - LoggerFactory.getLogger(RestActionStop.class); - - private final WebAppApi slider; - - public RestActionStop(WebAppApi slider) { - this.slider = slider; - } - - public StopResponse stop(HttpServletRequest request, UriInfo uriInfo, String body) { - String verb = request.getMethod(); - log.info("Ping {}", verb); - StopResponse response = new StopResponse(); - response.verb = verb; - long time = System.currentTimeMillis(); - String text = - String.format(Locale.ENGLISH, - "Stopping action %s received at %tc", - verb, time); - response.text = text; - log.info(text); - ActionStopSlider stopSlider = - new ActionStopSlider(text, - 1000, - TimeUnit.MILLISECONDS, - LauncherExitCodes.EXIT_SUCCESS, - FinalApplicationStatus.SUCCEEDED, - text); - log.info("SliderAppMasterApi.stopCluster: {}", stopSlider); - slider.getQueues().schedule(stopSlider); - - return response; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/StopResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/StopResponse.java deleted file mode 100644 index d591f57..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/StopResponse.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest.application.actions; - -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.annotate.JsonSerialize; - -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL) -public class StopResponse { - public String verb; - public String text; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/package-info.java deleted file mode 100644 index 34eb82c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/package-info.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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 package contains resources related to the application - * REST API - */ -package org.apache.slider.server.appmaster.web.rest.application; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java deleted file mode 100644 index 22fd0fe..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest.application.resources; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.util.Time; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A cached resource is one that can be stored and served up, with a refresh - * only taking place when the expiry happens. - * - * The refresh check/refresh is synchronized. - * @param type to return - */ -public class CachedContent { - private static final Logger log = - LoggerFactory.getLogger(CachedContent.class); - private T cachedValue; - private long expires; - private final long lifespan; - private final ResourceRefresher refresh; - private int refreshCounter; - - public CachedContent(long lifespan, - ResourceRefresher refresh) { - this.lifespan = lifespan; - this.refresh = refresh; - } - - /** - * Get the value, triggering a refresh if needed - * @return the cached or latest value. - */ - public T get() throws Exception { - maybeRefresh(); - return getCachedValue(); - } - - /** - * Get the cached value without any expiry check - * @return the last value set. May be null. - */ - public synchronized T getCachedValue() { - return cachedValue; - } - - public synchronized int getRefreshCounter() { - return refreshCounter; - } - - /** - * Get the lifespan in millis of the cached value - * @return the lifespan - */ - public long getLifespan() { - return lifespan; - } - - /** - * Maybe refresh the content - * @return true if a refresh took place. - */ - public synchronized boolean maybeRefresh() throws Exception { - long now = now(); - if (cachedValue == null || now >= expires) { - log.debug("Refreshing at time {}", now); - forceRefresh(); - log.debug("Refreshed value now {}", cachedValue); - return true; - } - return false; - } - - protected long now() { - return Time.monotonicNow(); - } - - /** - * Force a refresh and reset the expiry counter - * @return the new value - */ - protected synchronized T forceRefresh() throws Exception { - refreshCounter ++; - T updated = refresh.refresh(); - Preconditions.checkNotNull(updated); - cachedValue = updated; - expires = now() + lifespan; - return cachedValue; - } - - @Override - public String toString() { - final StringBuilder sb = - new StringBuilder("CachedContent{"); - sb.append(" expires=").append(expires); - sb.append(", lifespan=").append(lifespan); - sb.append(", refresh=").append(refresh); - sb.append(", refreshCounter=").append(refreshCounter); - sb.append(", cached=").append(cachedValue); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java deleted file mode 100644 index 8f026a1..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest.application.resources; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.concurrent.ConcurrentHashMap; - -/** - * Cache of content - */ -public class ContentCache extends ConcurrentHashMap { - - public ContentCache(int initialCapacity) { - super(initialCapacity); - } - - public ContentCache() { - } - - - public Object lookup(String key) throws Exception { - CachedContent content = get(key); - if (content == null) { - throw new FileNotFoundException("no content for path " + key); - } - return content.get(); - } - - - /** - * Lookup a cached item. If an exception is raised on the refresh... - *
    - *
  1. IOExceptions are thrown directly
  2. - *
  3. Other exceptions are wrapped with an IOExceptions
  4. - *
- * @param key - * @return - * @throws IOException - */ - public Object lookupWithIOE(String key) throws IOException { - try { - return lookup(key); - } catch (IOException e) { - throw e; - } catch (Exception e) { - throw new IOException("Looking up " + key + ": " + e, e); - } - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java deleted file mode 100644 index 35f0367..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest.application.resources; - -/** - * Interface which must be implemented to act as a source for cached content. - * @param type to return - */ -public interface ResourceRefresher { - /** - * Build an up to date version of the data - * @return a new instance of the (JSON serializable) data - */ - T refresh() throws Exception; -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java deleted file mode 100644 index 3e9b764..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java +++ /dev/null @@ -1,271 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest.publisher; - -import org.apache.hadoop.yarn.webapp.NotFoundException; -import org.apache.slider.core.registry.docstore.ConfigFormat; -import org.apache.slider.core.registry.docstore.PublishedConfigSet; -import org.apache.slider.core.registry.docstore.PublishedConfiguration; -import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter; -import org.apache.slider.core.registry.docstore.PublishedExports; -import org.apache.slider.core.registry.docstore.PublishedExportsSet; -import org.apache.slider.core.registry.docstore.UriMap; -import org.apache.slider.server.appmaster.state.StateAccessForProviders; -import org.apache.slider.server.appmaster.web.WebAppApi; -import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.servlet.http.HttpServletResponse; -import javax.ws.rs.GET; -import javax.ws.rs.Path; -import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.UriInfo; -import java.io.IOException; -import java.net.URL; -import java.net.URLClassLoader; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.slider.server.appmaster.web.rest.RestPaths.PUBLISHED_CONFIGURATION_REGEXP; -import static org.apache.slider.server.appmaster.web.rest.RestPaths.PUBLISHED_CONFIGURATION_SET_REGEXP; - -/** - * This publishes configuration sets - */ -public class PublisherResource extends AbstractSliderResource { - protected static final Logger log = - LoggerFactory.getLogger(PublisherResource.class); - public static final String EXPORTS_NAME = "exports"; - public static final String EXPORTS_RESOURCES_PATH = "/" + EXPORTS_NAME; - public static final String EXPORT_RESOURCE_PATH = EXPORTS_RESOURCES_PATH + "/{exportname}" ; - public static final String SET_NAME = - "{setname: " + PUBLISHED_CONFIGURATION_SET_REGEXP + "}"; - public static final String SETNAME = "setname"; - public static final String CLASSPATH = "/classpath"; - public static final String CONFIG = "config"; - - public static final String SETNAME_PATTERN = - "{"+ SETNAME+": " + PUBLISHED_CONFIGURATION_SET_REGEXP + "}"; - private static final String CONFIG_PATTERN = - SETNAME_PATTERN + "/{"+ CONFIG +": " + PUBLISHED_CONFIGURATION_REGEXP + "}"; - private final StateAccessForProviders appState; - - public PublisherResource(WebAppApi slider) { - super(slider); - appState = slider.getAppState(); - } - - private void init(HttpServletResponse res, UriInfo uriInfo) { - res.setContentType(null); - log.debug(uriInfo.getRequestUri().toString()); - } - - /** - * Get a named config set - * @param setname name of the config set - * @return the config set - * @throws NotFoundException if there was no matching set - */ - private PublishedConfigSet getConfigSet(String setname) { - PublishedConfigSet configSet = - appState.getPublishedConfigSet(setname); - if (configSet == null) { - throw new NotFoundException("Not found: " + setname); - } - return configSet; - } - - @GET - @Path("/") - @Produces({MediaType.APPLICATION_JSON}) - public UriMap enumConfigSets( - @Context UriInfo uriInfo, - @Context HttpServletResponse res) { - init(res, uriInfo); - String baseURL = uriInfo.getRequestUri().toString(); - if (!baseURL.endsWith("/")) { - baseURL += "/"; - } - UriMap uriMap = new UriMap(); - for (String name : appState.listConfigSets()) { - uriMap.put(name, baseURL + name); - log.info("registering config set {} at {}", name, baseURL); - } - uriMap.put(EXPORTS_NAME, baseURL + EXPORTS_NAME); - return uriMap; - } - - @GET - @Path(CLASSPATH) - @Produces({MediaType.APPLICATION_JSON}) - public List getAMClassPath() { - URL[] urls = ((URLClassLoader) getClass().getClassLoader()).getURLs(); - return Arrays.asList(urls); - } - - @GET - @Path(EXPORTS_RESOURCES_PATH) - @Produces({MediaType.APPLICATION_JSON}) - public PublishedExportsSet gePublishedExports() { - - return appState.getPublishedExportsSet(); - } - - @GET - @Path(EXPORT_RESOURCE_PATH) - @Produces({MediaType.APPLICATION_JSON}) - public PublishedExports getAMExports2(@PathParam("exportname") String exportname, - @Context UriInfo uriInfo, - @Context HttpServletResponse res) { - init(res, uriInfo); - PublishedExportsSet set = appState.getPublishedExportsSet(); - return set.get(exportname); - } - - @GET - @Path("/"+ SETNAME_PATTERN) - @Produces({MediaType.APPLICATION_JSON}) - public PublishedConfigSet getPublishedConfiguration( - @PathParam(SETNAME) String setname, - @Context UriInfo uriInfo, - @Context HttpServletResponse res) { - init(res, uriInfo); - - logRequest(uriInfo); - PublishedConfigSet publishedConfigSet = getConfigSet(setname); - log.debug("Number of configurations: {}", publishedConfigSet.size()); - return publishedConfigSet.shallowCopy(); - } - - private void logRequest(UriInfo uriInfo) { - log.info(uriInfo.getRequestUri().toString()); - } - - @GET - @Path("/" + CONFIG_PATTERN) - @Produces({MediaType.APPLICATION_JSON}) - public PublishedConfiguration getConfigurationInstance( - @PathParam(SETNAME) String setname, - @PathParam(CONFIG) String config, - @Context UriInfo uriInfo, - @Context HttpServletResponse res) { - init(res, uriInfo); - - PublishedConfiguration publishedConfig = - getPublishedConfiguration(setname, config); - if (publishedConfig == null) { - log.info("Configuration {} not found", config); - throw new NotFoundException("Not found: " + uriInfo.getAbsolutePath()); - } - return publishedConfig; - } - - /** - * Get a configuration - * @param setname name of the config set - * @param config config - * @return null if there was a config, but not a set - * @throws NotFoundException if there was no matching set - */ - public PublishedConfiguration getPublishedConfiguration(String setname, - String config) { - return getConfigSet(setname).get(config); - } - - @GET - @Path("/" + CONFIG_PATTERN + ".json") - @Produces({MediaType.APPLICATION_JSON}) - public String getConfigurationContentJson( - @PathParam(SETNAME) String setname, - - @PathParam(CONFIG) String config, - @Context UriInfo uriInfo, - @Context HttpServletResponse res) throws IOException { - return getStringRepresentation(setname, config, uriInfo, res, - ConfigFormat.JSON); - } - - @GET - @Path("/" + CONFIG_PATTERN + ".xml") - @Produces({MediaType.APPLICATION_XML}) - public String getConfigurationContentXML( - @PathParam(SETNAME) String setname, - @PathParam(CONFIG) String config, - @Context UriInfo uriInfo, - @Context HttpServletResponse res) throws IOException { - return getStringRepresentation(setname, config, uriInfo, res, - ConfigFormat.XML); - } - - @GET - @Path("/" + CONFIG_PATTERN + ".properties") - @Produces({MediaType.APPLICATION_XML}) - public String getConfigurationContentProperties( - @PathParam(SETNAME) String setname, - - @PathParam(CONFIG) String config, - @Context UriInfo uriInfo, - @Context HttpServletResponse res) throws IOException { - - return getStringRepresentation(setname, config, uriInfo, res, - ConfigFormat.PROPERTIES); - } - - public String getStringRepresentation(String setname, - String config, - UriInfo uriInfo, - HttpServletResponse res, ConfigFormat format) throws IOException { - // delegate (including init) - PublishedConfiguration publishedConfig = - getConfigurationInstance(setname, config, uriInfo, res); - PublishedConfigurationOutputter outputter = - publishedConfig.createOutputter(format); - return outputter.asString(); - } - - @GET - @Path("/" + CONFIG_PATTERN +"/{propertyName}") - @Produces({MediaType.APPLICATION_JSON}) - public Map getConfigurationProperty( - @PathParam(SETNAME) String setname, - @PathParam(CONFIG) String config, - @PathParam("propertyName") String propertyName, - @Context UriInfo uriInfo, - @Context HttpServletResponse res) { - PublishedConfiguration publishedConfig = - getConfigurationInstance(setname, config, uriInfo, res); - String propVal = publishedConfig.entries.get(propertyName); - if (propVal == null) { - log.debug("Configuration property {} not found in configuration {}", - propertyName, config); - throw new NotFoundException("Property not found: " + propertyName); - } - Map rtnVal = new HashMap<>(); - rtnVal.put(propertyName, propVal); - - return rtnVal; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/PathEntryResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/PathEntryResource.java deleted file mode 100644 index efb09a8..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/PathEntryResource.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest.registry; - -import org.apache.hadoop.registry.client.types.ServiceRecord; -import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.map.annotate.JsonSerialize; - -import java.util.List; - -/** - * Representation of a path entry - */ -@JsonIgnoreProperties(ignoreUnknown = true) -@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL) -public class PathEntryResource { - - /** - * Child nodes: as the short path to each element - */ - public List nodes; - - /** - * Service record: if null —there is no resolvable service - * record at this node. - */ - public ServiceRecord service; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java deleted file mode 100644 index c824848..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java +++ /dev/null @@ -1,151 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest.registry; - -import com.google.inject.Singleton; -import org.apache.hadoop.fs.PathNotFoundException; -import org.apache.hadoop.registry.client.api.RegistryOperations; -import org.apache.hadoop.registry.client.exceptions.AuthenticationFailedException; -import org.apache.hadoop.registry.client.exceptions.InvalidRecordException; -import org.apache.hadoop.registry.client.exceptions.NoPathPermissionsException; -import org.apache.hadoop.registry.client.exceptions.NoRecordException; -import org.apache.hadoop.yarn.webapp.ForbiddenException; -import org.apache.hadoop.yarn.webapp.NotFoundException; -import org.apache.slider.server.appmaster.web.WebAppApi; -import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.GET; -import javax.ws.rs.Path; -import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.UriInfo; -import java.io.IOException; - -/** - * This is the read-only view of the YARN registry. - * - * Model: - *
    - *
  1. a tree of nodes
  2. - *
  3. Default view is of children + record
  4. - *
- * - */ -@Singleton -public class RegistryResource extends AbstractSliderResource { - protected static final Logger log = - LoggerFactory.getLogger(RegistryResource.class); - public static final String SERVICE_PATH = - "/{path:.*}"; - - private final RegistryOperations registry; - - /** - * Construct an instance bonded to a registry - * @param slider slider API - */ - public RegistryResource(WebAppApi slider) { - super(slider); - this.registry = slider.getRegistryOperations(); - } - - - /** - * Internal init code, per request - * @param request incoming request - * @param uriInfo URI details - */ - private void init(HttpServletRequest request, UriInfo uriInfo) { - log.debug(uriInfo.getRequestUri().toString()); - } - - @GET - @Produces({MediaType.APPLICATION_JSON}) - public PathEntryResource getRoot(@Context HttpServletRequest request, - @Context UriInfo uriInfo) { - return lookup("/", request, uriInfo); - } - -// {path:.*} - - @Path(SERVICE_PATH) - @GET - @Produces({MediaType.APPLICATION_JSON}) - public PathEntryResource lookup( - @PathParam("path") String path, - @Context HttpServletRequest request, - @Context UriInfo uriInfo) { - init(request, uriInfo); - return resolvePath(path); - } - - /** - * Do the actual processing of requests to responses; can be directly - * invoked for testing. - * @param path path to query - * @return the entry - * @throws WebApplicationException on any failure. - */ - public PathEntryResource resolvePath(String path) throws - WebApplicationException { - try { - PathEntryResource pathEntry = - fromRegistry(path); - if (log.isDebugEnabled()) { - log.debug("Resolved:\n{}", pathEntry); - } - return pathEntry; - - } catch (Exception e) { - throw buildException(path, e); - } - } - - - /** - * Build from the registry, filling up the children and service records. - * If there is no service record at the end of the path, that entry is - * null - * @param path path to query - * @return the built up record - * @throws IOException problems - * - */ - private PathEntryResource fromRegistry(String path) throws IOException { - PathEntryResource entry = new PathEntryResource(); - try { - entry.service = registry.resolve(path); - } catch (NoRecordException e) { - // ignoring - log.debug("No record at {}", path); - } catch (InvalidRecordException e) { - // swallowing this exception, the sign of "no entry present" - // "nothing parseable" - log.warn("Failed to resolve {}: {}", path, e, e); - } - entry.nodes = registry.list(path); - return entry; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ClusterSpecificationBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ClusterSpecificationBlock.java deleted file mode 100644 index 79b687f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ClusterSpecificationBlock.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.slider.server.appmaster.web.view; - -import com.google.inject.Inject; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; -import org.apache.slider.server.appmaster.web.WebAppApi; - -/** - * - */ -public class ClusterSpecificationBlock extends SliderHamletBlock { - - @Inject - public ClusterSpecificationBlock(WebAppApi slider) { - super(slider); - } - - @Override - protected void render(Block html) { - doRender(html); - } - - // An extra method to make testing easier since you can't make an instance of Block - protected void doRender(Hamlet html) { - html. - div("cluster_json"). - h2("JSON Cluster Specification"). - pre(). - _(getJson())._()._(); - } - - /** - * Get the JSON, catching any exceptions and returning error text instead - * @return - */ - private String getJson() { - return appState.getApplication().toString(); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java deleted file mode 100644 index 4796d6c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java +++ /dev/null @@ -1,275 +0,0 @@ -/* - * 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.slider.server.appmaster.web.view; - -import com.google.common.base.Function; -import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; -import com.google.inject.Inject; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR; -import org.apache.slider.api.ClusterNode; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.types.ComponentInformation; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.web.WebAppApi; - -import javax.annotation.Nonnull; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - -/** - * - */ -public class ContainerStatsBlock extends SliderHamletBlock { - - private static final String EVEN = "even", ODD = "odd", BOLD = "bold", SCHEME = "http://", PATH = "/node/container/"; - - // Some functions that help transform the data into an object we can use to abstract presentation specifics - protected static final Function,Entry> stringIntPairFunc = toTableContentFunction(); - protected static final Function,Entry> stringLongPairFunc = toTableContentFunction(); - protected static final Function,Entry> stringStringPairFunc = toTableContentFunction(); - - @Inject - public ContainerStatsBlock(WebAppApi slider) { - super(slider); - } - - /** - * Sort a collection of ClusterNodes by name - */ - protected static class ClusterNodeNameComparator implements Comparator, - Serializable { - - @Override - public int compare(ClusterNode node1, ClusterNode node2) { - if (null == node1 && null != node2) { - return -1; - } else if (null != node1 && null == node2) { - return 1; - } else if (null == node1) { - return 0; - } - - final String name1 = node1.name, name2 = node2.name; - if (null == name1 && null != name2) { - return -1; - } else if (null != name1 && null == name2) { - return 1; - } else if (null == name1) { - return 0; - } - - return name1.compareTo(name2); - } - - } - - @Override - protected void render(Block html) { - final Map containerInstances = getContainerInstances( - appState.cloneOwnedContainerList()); - - Map> clusterNodeMap = - appState.getRoleClusterNodeMapping(); - Map componentInfoMap = appState.getComponentInfoSnapshot(); - - for (Entry> entry : clusterNodeMap.entrySet()) { - final String name = entry.getKey(); - Map clusterNodesInRole = entry.getValue(); - //final RoleStatus roleStatus = entry.getValue(); - - DIV div = html.div("role-info ui-widget-content ui-corner-all"); - - List nodesInRole = - new ArrayList<>(clusterNodesInRole.values()); - - div.h2(BOLD, StringUtils.capitalize(name)); - - // Generate the details on this role - ComponentInformation componentInfo = componentInfoMap.get(name); - if (componentInfo != null) { - Iterable> stats = componentInfo.buildStatistics().entrySet(); - generateRoleDetails(div,"role-stats-wrap", "Specifications", - Iterables.transform(stats, stringIntPairFunc)); - } - - // Sort the ClusterNodes by their name (containerid) - Collections.sort(nodesInRole, new ClusterNodeNameComparator()); - - // Generate the containers running this role - generateRoleDetails(div, "role-stats-containers", "Containers", - Iterables.transform(nodesInRole, new Function>() { - - @Override - public Entry apply(ClusterNode input) { - final String containerId = input.name; - - if (containerInstances.containsKey(containerId)) { - RoleInstance roleInst = containerInstances.get(containerId); - if (roleInst.container.getNodeHttpAddress() != null) { - return Maps. immutableEntry( - new TableAnchorContent(containerId, - buildNodeUrlForContainer(roleInst.container.getNodeHttpAddress(), containerId)), null); - } - } - return Maps.immutableEntry(new TableContent(input.name), null); - } - - })); - - Application application = appState.getApplication(); - Iterable> tableContent; - tableContent = Collections.emptySet(); - - // Generate the options used by this role - generateRoleDetails(div, "role-options-wrap", "Role Options", tableContent); - - // Close the div for this role - div._(); - } - } - - protected static Function,Entry> toTableContentFunction() { - return new Function,Entry>() { - @Override - public Entry apply(@Nonnull Entry input) { - return Maps.immutableEntry(new TableContent(input.getKey()), input.getValue()); - } - }; - } - - protected Map getContainerInstances(List roleInstances) { - Map map = Maps.newHashMapWithExpectedSize(roleInstances.size()); - for (RoleInstance roleInstance : roleInstances) { - // UUID is the containerId - map.put(roleInstance.id, roleInstance); - } - return map; - } - - /** - * Given a div, a name for this data, and some pairs of data, generate a nice HTML table. If contents is empty (of size zero), then a mesage will be printed - * that there were no items instead of an empty table. - * - */ - protected void generateRoleDetails(DIV parent, String divSelector, String detailsName, Iterable> contents) { - final DIV> div = parent.div(divSelector).h3(BOLD, detailsName); - - int offset = 0; - TABLE>> table = null; - TBODY>>> tbody = null; - for (Entry content : contents) { - if (null == table) { - table = div.table("ui-widget-content ui-corner-bottom"); - tbody = table.tbody(); - } - - TR>>>> row = tbody.tr(offset % 2 == 0 ? EVEN : ODD); - - // Defer to the implementation of the TableContent for what the cell should contain - content.getKey().printCell(row); - - // Only add the second column if the element is non-null - // This also lets us avoid making a second method if we're only making a one-column table - if (null != content.getValue()) { - row.td(content.getValue().toString()); - } - - row._(); - - offset++; - } - - // If we made a table, close it out - if (null != table) { - tbody._()._(); - } else { - // Otherwise, throw in a nice "no content" message - div.p("no-table-contents")._("None")._(); - } - - // Close out the initial div - div._(); - } - - /** - * Build a URL from the address:port and container ID directly to the NodeManager service - * @param nodeAddress - * @param containerId - * @return - */ - protected String buildNodeUrlForContainer(String nodeAddress, String containerId) { - StringBuilder sb = new StringBuilder(SCHEME.length() + nodeAddress.length() + PATH.length() + containerId.length()); - - sb.append(SCHEME).append(nodeAddress).append(PATH).append(containerId); - - return sb.toString(); - } - - /** - * Creates a table cell with the provided String as content. - */ - protected static class TableContent { - private String cell; - - public TableContent(String cell) { - this.cell = cell; - } - - public String getCell() { - return cell; - } - - /** - * Adds a td to the given tr. The tr is not closed - * @param tableRow - */ - public void printCell(TR tableRow) { - tableRow.td(this.cell); - } - } - - /** - * Creates a table cell with an anchor to the given URL with the provided String as content. - */ - protected static class TableAnchorContent extends TableContent { - private String anchorUrl; - - public TableAnchorContent(String cell, String anchorUrl) { - super(cell); - this.anchorUrl = anchorUrl; - } - - /* (non-javadoc) - * @see org.apache.slider.server.appmaster.web.view.ContainerStatsBlock$TableContent#printCell() - */ - @Override - public void printCell(TR tableRow) { - tableRow.td().a(anchorUrl, getCell())._(); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java deleted file mode 100644 index c0a120d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java +++ /dev/null @@ -1,273 +0,0 @@ -/* - * 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.slider.server.appmaster.web.view; - -import com.google.common.annotations.VisibleForTesting; -import com.google.inject.Inject; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.LI; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.UL; -import org.apache.slider.api.types.ApplicationLivenessInformation; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.registry.docstore.ExportEntry; -import org.apache.slider.core.registry.docstore.PublishedExports; -import org.apache.slider.core.registry.docstore.PublishedExportsSet; -import org.apache.hadoop.yarn.service.metrics.ServiceMetrics; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.apache.slider.server.appmaster.web.WebAppApi; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map.Entry; -import java.util.Set; - -import static org.apache.slider.server.appmaster.web.rest.RestPaths.LIVE_COMPONENTS; - -/** - * The main content on the Slider AM web page - */ -public class IndexBlock extends SliderHamletBlock { - private static final Logger log = LoggerFactory.getLogger(IndexBlock.class); - - /** - * Message printed when application is at full size. - * - * {@value} - */ - public static final String ALL_CONTAINERS_ALLOCATED = "all containers allocated"; - - @Inject - public IndexBlock(WebAppApi slider) { - super(slider); - } - - @Override - protected void render(Block html) { - doIndex(html, getProviderName()); - } - - // An extra method to make testing easier since you can't make an instance of Block - @VisibleForTesting - protected void doIndex(Hamlet html, String providerName) { - String name = appState.getApplicationName(); - if (name != null && (name.startsWith(" ") || name.endsWith(" "))) { - name = "'" + name + "'"; - } - DIV div = html.div("general_info") - .h1("index_header", - "Application: " + name); - - ApplicationLivenessInformation liveness = - appState.getApplicationLivenessInformation(); - String livestatus = liveness.allRequestsSatisfied - ? ALL_CONTAINERS_ALLOCATED - : String.format("Awaiting %d containers", liveness.requestsOutstanding); - Hamlet.TABLE> table1 = div.table(); - table1.tr() - .td("Status") - .td(livestatus) - ._(); - table1.tr() - .td("Total number of containers") - .td(Integer.toString(appState.getNumOwnedContainers())) - ._(); - table1.tr() - .td("Create time: ") - .td("N/A") - ._(); - table1.tr() - .td("Running since: ") - .td("N/A") - ._(); - table1.tr() - .td("Time last flexed: ") - .td("N/A") - ._(); - table1.tr() - .td("Application storage path: ") - .td("N/A") - ._(); - table1.tr() - .td("Application configuration path: ") - .td("N/A") - ._(); - table1._(); - div._(); - div = null; - - DIV containers = html.div("container_instances") - .h3("Component Instances"); - - int aaRoleWithNoSuitableLocations = 0; - int aaRoleWithOpenRequest = 0; - int roleWithOpenRequest = 0; - - Hamlet.TABLE> table = containers.table(); - Hamlet.TR>>> header = table.thead().tr(); - trb(header, "Component"); - trb(header, "Desired"); - trb(header, "Actual"); - trb(header, "Outstanding Requests"); - trb(header, "Failed"); - trb(header, "Failed to start"); - trb(header, "Placement"); - header._()._(); // tr & thead - - List roleStatuses = - new ArrayList<>(appState.getRoleStatusMap().values()); - Collections.sort(roleStatuses, new RoleStatus.CompareByName()); - for (RoleStatus status : roleStatuses) { - String roleName = status.getName(); - String nameUrl = apiPath(LIVE_COMPONENTS) + "/" + roleName; - String aatext; - if (status.isAntiAffinePlacement()) { - boolean aaRequestOutstanding = status.isAARequestOutstanding(); - int pending = (int)status.getAAPending(); - aatext = buildAADetails(aaRequestOutstanding, pending); - if (SliderUtils.isSet(status.getLabelExpression())) { - aatext += " (label: " + status.getLabelExpression() + ")"; - } - if (pending > 0 && !aaRequestOutstanding) { - aaRoleWithNoSuitableLocations ++; - } else if (aaRequestOutstanding) { - aaRoleWithOpenRequest++; - } - } else { - if (SliderUtils.isSet(status.getLabelExpression())) { - aatext = "label: " + status.getLabelExpression(); - } else { - aatext = ""; - } - if (status.getRequested() > 0) { - roleWithOpenRequest ++; - } - } - ServiceMetrics metrics = status.getComponentMetrics(); - table.tr() - .td().a(nameUrl, roleName)._() - .td(String.format("%d", metrics.containersDesired.value())) - .td(String.format("%d", metrics.containersRunning.value())) - .td(String.format("%d", metrics.containersRequested.value())) - .td(String.format("%d", metrics.containersFailed.value())) - .td(aatext) - ._(); - } - - // empty row for some more spacing - table.tr()._(); - // close table - table._(); - - containers._(); - containers = null; - - // some spacing - html.div()._(); - html.div()._(); - - DIV diagnostics = html.div("diagnostics"); - - List statusEntries = new ArrayList<>(0); - if (roleWithOpenRequest > 0) { - statusEntries.add(String.format("%d %s with requests unsatisfiable by cluster", - roleWithOpenRequest, plural(roleWithOpenRequest, "component"))); - } - if (aaRoleWithNoSuitableLocations > 0) { - statusEntries.add(String.format("%d anti-affinity %s no suitable nodes in the cluster", - aaRoleWithNoSuitableLocations, - plural(aaRoleWithNoSuitableLocations, "component has", "components have"))); - } - if (aaRoleWithOpenRequest > 0) { - statusEntries.add(String.format("%d anti-affinity %s with requests unsatisfiable by cluster", - aaRoleWithOpenRequest, - plural(aaRoleWithOpenRequest, "component has", "components have"))); - - } - if (!statusEntries.isEmpty()) { - diagnostics.h3("Diagnostics"); - Hamlet.TABLE> diagnosticsTable = diagnostics.table(); - for (String entry : statusEntries) { - diagnosticsTable.tr().td(entry)._(); - } - diagnosticsTable._(); - } - diagnostics._(); - - DIV provider_info = html.div("provider_info"); - provider_info.h3(providerName + " information"); - UL ul = html.ul(); - //TODO render app/cluster status - ul._(); - provider_info._(); - - DIV exports = html.div("exports"); - exports.h3("Exports"); - ul = html.ul(); - enumeratePublishedExports(appState.getPublishedExportsSet(), ul); - ul._(); - exports._(); - } - - @VisibleForTesting - String buildAADetails(boolean outstanding, int pending) { - return String.format("Anti-affinity:%s %d pending %s", - (outstanding ? " 1 active request and" : ""), - pending, plural(pending, "request")); - } - - private String plural(int n, String singular) { - return plural(n, singular, singular + "s"); - } - private String plural(int n, String singular, String plural) { - return n == 1 ? singular : plural; - } - - private void trb(Hamlet.TR tr, - String text) { - tr.td().b(text)._(); - } - - private String getProviderName() { - return "docker"; - } - - - protected void enumeratePublishedExports(PublishedExportsSet exports, UL ul) { - for(String key : exports.keys()) { - PublishedExports export = exports.get(key); - LI> item = ul.li(); - item.span().$class("bold")._(export.description)._(); - UL sublist = item.ul(); - for (Entry> entry : export.sortedEntries() - .entrySet()) { - if (SliderUtils.isNotEmpty(entry.getValue())) { - LI sublistItem = sublist.li()._(entry.getKey()); - for (ExportEntry exportEntry : entry.getValue()) { - sublistItem._(exportEntry.getValue()); - } - sublistItem._(); - } - } - sublist._(); - item._(); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/NavBlock.java deleted file mode 100644 index 069d386..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/NavBlock.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.slider.server.appmaster.web.view; - -import com.google.inject.Inject; -import org.apache.slider.server.appmaster.web.WebAppApi; - -import static org.apache.slider.server.appmaster.web.SliderAMWebApp.*; -import static org.apache.slider.server.appmaster.web.rest.RestPaths.*; - -/** - * - */ -public class NavBlock extends SliderHamletBlock { - - @Inject - public NavBlock(WebAppApi slider) { - super(slider); - } - - @Override - protected void render(Block html) { - html. - div("#nav"). - h3("Slider"). - ul(). - li().a(this.prefix(), "Overview")._(). - li().a(relPath(CONTAINER_STATS), "Statistics")._(). - li().a(relPath(CLUSTER_SPEC), "Specification")._(). - li().a(rootPath(SYSTEM_METRICS_JSON), "Metrics")._(). - li().a(rootPath(SYSTEM_HEALTHCHECK), "Health")._(). - li().a(rootPath(SYSTEM_THREADS), "Threads")._(). - _() - .h3("REST API"). - ul(). - li().a(apiPath(MODEL_DESIRED), "Specified")._(). - li().a(apiPath(MODEL_RESOLVED), "Resolved")._(). - li().a(apiPath(LIVE_RESOURCES), "Resources")._(). - li().a(apiPath(LIVE_COMPONENTS), "Components")._(). - li().a(apiPath(LIVE_CONTAINERS), "Containers")._(). - li().a(apiPath(LIVE_NODES), "Nodes")._(). - li().a(apiPath(LIVE_STATISTICS), "Statistics")._(). - li().a(apiPath(LIVE_LIVENESS), "Liveness")._() - ._() - ._(); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/SliderHamletBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/SliderHamletBlock.java deleted file mode 100644 index 5f44bda..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/SliderHamletBlock.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.slider.server.appmaster.web.view; - -import org.apache.hadoop.yarn.webapp.view.HtmlBlock; -import org.apache.slider.server.appmaster.state.StateAccessForProviders; -import org.apache.slider.server.appmaster.web.WebAppApi; -import org.apache.slider.server.appmaster.web.rest.RestPaths; - -import static org.apache.hadoop.yarn.util.StringHelper.ujoin; -import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_APPLICATION; - -/** - * Anything we want to share across slider hamlet blocks - */ -public abstract class SliderHamletBlock extends HtmlBlock { - - protected final StateAccessForProviders appState; - protected final RestPaths restPaths = new RestPaths(); - - public SliderHamletBlock(WebAppApi slider) { - this.appState = slider.getAppState(); - } - - protected String rootPath(String absolutePath) { - return root_url(absolutePath); - } - - protected String relPath(String... args) { - return ujoin(this.prefix(), args); - } - - protected String apiPath(String api) { - return root_url(SLIDER_PATH_APPLICATION, api); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/LoadedRoleHistory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/LoadedRoleHistory.java deleted file mode 100644 index 77408a5..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/LoadedRoleHistory.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * 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.slider.server.avro; - -import org.apache.hadoop.fs.Path; -import org.apache.slider.common.tools.SliderUtils; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * The role history - */ -public class LoadedRoleHistory { - - private RoleHistoryHeader header; - - private Path path; - - public final Map roleMap = new HashMap<>(); - - public final List records = new ArrayList<>(); - - /** - * Add a record - * @param record - */ - public void add(NodeEntryRecord record) { - records.add(record); - } - - /** - * Number of loaded records - * @return - */ - public int size() { - return records.size(); - } - - public RoleHistoryHeader getHeader() { - return header; - } - - public void setHeader(RoleHistoryHeader header) { - this.header = header; - } - - public Path getPath() { - return path; - } - - public void setPath(Path path) { - this.path = path; - } - - public void buildMapping(Map source) { - roleMap.clear(); - for (Map.Entry entry : source.entrySet()) { - roleMap.put(SliderUtils.sequenceToString(entry.getKey()), - entry.getValue()); - } - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder( - "LoadedRoleHistory{"); - sb.append("path=").append(path); - sb.append("; number of roles=").append(roleMap.size()); - sb.append("; size=").append(size()); - sb.append('}'); - return sb.toString(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/NewerFilesFirst.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/NewerFilesFirst.java deleted file mode 100644 index 2e049cb..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/NewerFilesFirst.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.slider.server.avro; - -import org.apache.hadoop.fs.Path; - -import java.io.Serializable; -import java.util.Comparator; - -/** - * Compare two filenames by name; the more recent one comes first - */ -public class NewerFilesFirst implements Comparator, Serializable { - - /** - * Takes the ordering of path names from the normal string comparison - * and negates it, so that names that come after other names in - * the string sort come before here - * @param o1 leftmost - * @param o2 rightmost - * @return positive if o1 > o2 - */ - @Override - public int compare(Path o1, Path o2) { - return (o2.getName().compareTo(o1.getName())); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/OlderFilesFirst.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/OlderFilesFirst.java deleted file mode 100644 index 407aaa6..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/OlderFilesFirst.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.slider.server.avro; - -import org.apache.hadoop.fs.Path; - -import java.io.Serializable; -import java.util.Comparator; - -/** - * Compare two filenames by name; the older ones comes first - */ -public class OlderFilesFirst implements Comparator, Serializable { - - /** - * Takes the ordering of path names from the normal string comparison - * and negates it, so that names that come after other names in - * the string sort come before here - * @param o1 leftmost - * @param o2 rightmost - * @return positive if o1 > o2 - */ - @Override - public int compare(Path o1, Path o2) { - return (o1.getName().compareTo(o2.getName())); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/RoleHistoryWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/RoleHistoryWriter.java deleted file mode 100644 index 52553d0..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/RoleHistoryWriter.java +++ /dev/null @@ -1,449 +0,0 @@ -/* - * 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.slider.server.avro; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.avro.AvroTypeException; -import org.apache.avro.Schema; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.DatumWriter; -import org.apache.avro.io.Decoder; -import org.apache.avro.io.DecoderFactory; -import org.apache.avro.io.Encoder; -import org.apache.avro.io.EncoderFactory; -import org.apache.avro.specific.SpecificDatumReader; -import org.apache.avro.specific.SpecificDatumWriter; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.GlobFilter; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathFilter; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.server.appmaster.state.NodeEntry; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.RoleHistory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.EOFException; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.ListIterator; -import java.util.Locale; -import java.util.Map; - -/** - * Write out the role history to an output stream. - */ -public class RoleHistoryWriter { - protected static final Logger log = - LoggerFactory.getLogger(RoleHistoryWriter.class); - - /** - * Although Avro is designed to handle some changes, we still keep a version - * marker in the file to catch changes that are fundamentally incompatible - * at the semantic level -changes that require either a different - * parser or get rejected outright. - */ - public static final int ROLE_HISTORY_VERSION = 0x01; - - /** - * Write out the history. - * This does not update the history's dirty/savetime fields - * - * @param out outstream - * @param history history - * @param savetime time in millis for the save time to go in as a record - * @return no of records written - * @throws IOException IO failures - */ - public long write(OutputStream out, RoleHistory history, long savetime) - throws IOException { - try { - DatumWriter writer = - new SpecificDatumWriter<>(RoleHistoryRecord.class); - - RoleHistoryRecord record = createHeaderRecord(savetime, history); - int roles = history.getRoleSize(); - Schema schema = record.getSchema(); - Encoder encoder = EncoderFactory.get().jsonEncoder(schema, out); - writer.write(record, encoder); - // now write the rolemap record - writer.write(createRolemapRecord(history), encoder); - long count = 0; - //now for every role history entry, write out its record - Collection instances = history.cloneNodemap().values(); - for (NodeInstance instance : instances) { - for (int role = 0; role < roles; role++) { - NodeEntry nodeEntry = instance.get(role); - - if (nodeEntry != null) { - NodeEntryRecord ner = build(nodeEntry, role, instance.hostname); - record = new RoleHistoryRecord(ner); - writer.write(record, encoder); - count++; - } - } - } - // footer - RoleHistoryFooter footer = new RoleHistoryFooter(); - footer.setCount(count); - writer.write(new RoleHistoryRecord(footer), encoder); - encoder.flush(); - return count; - } finally { - out.close(); - } - } - - /** - * Create the header record - * @param savetime time of save - * @param history history - * @return a record to place at the head of the file - */ - private RoleHistoryRecord createHeaderRecord(long savetime, RoleHistory history) { - RoleHistoryHeader header = new RoleHistoryHeader(); - header.setVersion(ROLE_HISTORY_VERSION); - header.setSaved(savetime); - header.setSavedx(Long.toHexString(savetime)); - header.setSavedate(SliderUtils.toGMTString(savetime)); - header.setRoles(history.getRoleSize()); - return new RoleHistoryRecord(header); - } - - /** - * Create the rolemap record - * @param history history - * @return a record to insert into the file - */ - private RoleHistoryRecord createRolemapRecord(RoleHistory history) { - RoleHistoryMapping entry = new RoleHistoryMapping(); - Map mapping = history.buildMappingForHistoryFile(); - entry.setRolemap(mapping); - return new RoleHistoryRecord(entry); - } - - /** - * Write the history information to a file - * - * @param fs filesystem - * @param path path - * @param overwrite overwrite flag - * @param history history - * @param savetime time in millis for the save time to go in as a record - * @return no of records written - * @throws IOException IO failures - */ - public long write(FileSystem fs, - Path path, - boolean overwrite, - RoleHistory history, - long savetime) - throws IOException { - FSDataOutputStream out = fs.create(path, overwrite); - return write(out, history, savetime); - } - - - /** - * Create the filename for a history file - * @param time time value - * @return a filename such that later filenames sort later in the directory - */ - public Path createHistoryFilename(Path historyPath, long time) { - String filename = String.format(Locale.ENGLISH, - SliderKeys.HISTORY_FILENAME_CREATION_PATTERN, - time); - Path path = new Path(historyPath, filename); - return path; - } - - /** - * Build a {@link NodeEntryRecord} from a node entry; include whether - * the node is in use and when it was last used. - * @param entry entry count - * @param role role index - * @param hostname name - * @return the record - */ - private NodeEntryRecord build(NodeEntry entry, int role, String hostname) { - NodeEntryRecord record = new NodeEntryRecord( - hostname, role, entry.getLive() > 0, entry.getLastUsed() - ); - return record; - } - - /** - * Read a history, returning one that is ready to have its onThaw() - * method called - * @param in input source - * @return no. of entries read - * @throws IOException problems - */ - public LoadedRoleHistory read(InputStream in) throws - IOException, - BadConfigException { - try { - LoadedRoleHistory loadedRoleHistory = new LoadedRoleHistory(); - DatumReader reader = - new SpecificDatumReader<>(RoleHistoryRecord.class); - Decoder decoder = - DecoderFactory.get().jsonDecoder(RoleHistoryRecord.getClassSchema(), - in); - - //read header : no entry -> EOF - RoleHistoryRecord record = reader.read(null, decoder); - if (record == null) { - throw new IOException("Role History Header not found at start of file."); - } - Object entry = record.getEntry(); - if (!(entry instanceof RoleHistoryHeader)) { - throw new IOException("Role History Header not found at start of file"); - } - RoleHistoryHeader header = (RoleHistoryHeader) entry; - if (header.getVersion() != ROLE_HISTORY_VERSION) { - throw new IOException( - String.format("Can't read role file version %04x -need %04x", - header.getVersion(), - ROLE_HISTORY_VERSION)); - } - loadedRoleHistory.setHeader(header); - RoleHistoryFooter footer = null; - int records = 0; - //go through reading data - try { - while (footer == null) { - record = reader.read(null, decoder); - if (record == null) { - throw new IOException("Null record after " + records + " records"); - } - entry = record.getEntry(); - - if (entry instanceof RoleHistoryHeader) { - throw new IOException("Duplicate Role History Header found"); - } else if (entry instanceof RoleHistoryMapping) { - // role history mapping entry - if (!loadedRoleHistory.roleMap.isEmpty()) { - // duplicate role maps are viewed as something to warn over, rather than fail - log.warn("Duplicate role map; ignoring"); - } else { - RoleHistoryMapping historyMapping = (RoleHistoryMapping) entry; - loadedRoleHistory.buildMapping(historyMapping.getRolemap()); - } - } else if (entry instanceof NodeEntryRecord) { - // normal record - records++; - NodeEntryRecord nodeEntryRecord = (NodeEntryRecord) entry; - loadedRoleHistory.add(nodeEntryRecord); - } else if (entry instanceof RoleHistoryFooter) { - //tail end of the file - footer = (RoleHistoryFooter) entry; - } else { - // this is to handle future versions, such as when rolling back - // from a later version of slider - log.warn("Discarding unknown record {}", entry); - } - } - } catch (EOFException e) { - EOFException ex = new EOFException( - "End of file reached after " + records + " records"); - ex.initCause(e); - throw ex; - } - // at this point there should be no data left. - // check by reading and expecting a -1 - if (in.read() > 0) { - // footer is in stream before the last record - throw new EOFException( - "File footer reached before end of file -after " + records + - " records"); - } - if (records != footer.getCount()) { - log.warn("mismatch between no of records saved {} and number read {}", - footer.getCount(), records); - } - return loadedRoleHistory; - } finally { - in.close(); - } - - } - - /** - * Read a role history from a path in a filesystem - * @param fs filesystem - * @param path path to the file - * @return the records read - * @throws IOException any problem - */ - public LoadedRoleHistory read(FileSystem fs, Path path) - throws IOException, BadConfigException { - FSDataInputStream instream = fs.open(path); - return read(instream); - } - - /** - * Read from a resource in the classpath -used for testing - * @param resource resource - * @return the records read - * @throws IOException any problem - */ - public LoadedRoleHistory read(String resource) - throws IOException, BadConfigException { - - return read(this.getClass().getClassLoader().getResourceAsStream(resource)); - } - - - /** - * Find all history entries in a dir. The dir is created if it is - * not already defined. - * - * The scan uses the match pattern {@link SliderKeys#HISTORY_FILENAME_MATCH_PATTERN} - * while dropping empty files and directories which match the pattern. - * The list is then sorted with a comparator that sorts on filename, - * relying on the filename of newer created files being later than the old ones. - * - * - * - * @param fs filesystem - * @param dir dir to scan - * @param includeEmptyFiles should empty files be included in the result? - * @return a possibly empty list - * @throws IOException IO problems - * @throws FileNotFoundException if the target dir is actually a path - */ - public List findAllHistoryEntries(FileSystem fs, - Path dir, - boolean includeEmptyFiles) throws IOException { - assert fs != null; - assert dir != null; - if (!fs.exists(dir)) { - fs.mkdirs(dir); - } else if (!fs.isDirectory(dir)) { - throw new FileNotFoundException("Not a directory " + dir.toString()); - } - - PathFilter filter = new GlobFilter(SliderKeys.HISTORY_FILENAME_GLOB_PATTERN); - FileStatus[] stats = fs.listStatus(dir, filter); - List paths = new ArrayList(stats.length); - for (FileStatus stat : stats) { - log.debug("Possible entry: {}", stat.toString()); - if (stat.isFile() && (includeEmptyFiles || stat.getLen() > 0)) { - paths.add(stat.getPath()); - } - } - sortHistoryPaths(paths); - return paths; - } - - @VisibleForTesting - public static void sortHistoryPaths(List paths) { - Collections.sort(paths, new NewerFilesFirst()); - } - - /** - * Iterate through the paths until one can be loaded - * @param paths paths to load - * @return the loaded history including the path -or null if all failed to load - */ - public LoadedRoleHistory attemptToReadHistory(FileSystem fileSystem, - List paths) - throws BadConfigException { - ListIterator pathIterator = paths.listIterator(); - boolean success = false; - LoadedRoleHistory history = null; - while (!success && pathIterator.hasNext()) { - Path path = pathIterator.next(); - try { - history = read(fileSystem, path); - //success - success = true; - history.setPath(path); - } catch (IOException e) { - log.info("Failed to read {}", path, e); - } catch (AvroTypeException e) { - log.warn("Failed to parse {}", path, e); - } catch (Exception e) { - // low level event logged @ warn level - log.warn("Exception while reading {}", path, e); - } - } - return history; - } - - /** - * Try to load the history from a directory -a failure to load a specific - * file is downgraded to a log and the next older path attempted instead - * @param fs filesystem - * @param dir dir to load from - * @return the history loaded, including the path - * @throws IOException if indexing the history directory fails. - */ - public LoadedRoleHistory loadFromHistoryDir(FileSystem fs, Path dir) - throws IOException, BadConfigException { - assert fs != null: "null filesystem"; - List entries = findAllHistoryEntries(fs, dir, false); - return attemptToReadHistory(fs, entries); - } - - /** - * Delete all old history entries older than the one we want to keep. This - * uses the filename ordering to determine age, not timestamps - * @param fileSystem filesystem - * @param keep path to keep -used in thresholding the files - * @return the number of files deleted - * @throws FileNotFoundException if the path to keep is not present (safety - * check to stop the entire dir being purged) - * @throws IOException IO problems - */ - public int purgeOlderHistoryEntries(FileSystem fileSystem, Path keep) - throws IOException { assert fileSystem != null : "null filesystem"; - if (!fileSystem.exists(keep)) { - throw new FileNotFoundException(keep.toString()); - } - Path dir = keep.getParent(); - log.debug("Purging entries in {} up to {}", dir, keep); - List paths = findAllHistoryEntries(fileSystem, dir, true); - Collections.sort(paths, new OlderFilesFirst()); - int deleteCount = 0; - for (Path path : paths) { - if (path.equals(keep)) { - break; - } else { - log.debug("Deleting {}", path); - deleteCount++; - fileSystem.delete(path, false); - } - } - return deleteCount; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/AbstractSliderLaunchedService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/AbstractSliderLaunchedService.java deleted file mode 100644 index 43f0e4e..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/AbstractSliderLaunchedService.java +++ /dev/null @@ -1,118 +0,0 @@ -/* - * 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.slider.server.services.utility; - -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.yarn.conf.YarnConfiguration; -import org.apache.slider.common.tools.ConfigHelper; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.core.zk.ZookeeperUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Base service for the standard slider client/server services - */ -public abstract class AbstractSliderLaunchedService extends - LaunchedWorkflowCompositeService { - private static final Logger log = - LoggerFactory.getLogger(AbstractSliderLaunchedService.class); - - protected AbstractSliderLaunchedService(String name) { - super(name); - // make sure all the yarn configs get loaded - ConfigHelper.registerDeprecatedConfigItems(); - } - - /** - * look up the registry quorum from the config - * @return the quorum string - * @throws BadConfigException if it is not there or invalid - */ - public String lookupZKQuorum() 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); - return registryQuorum; - } - - /** - * Create, adopt ,and start the YARN registration service - * @return the registry operations service, already deployed as a child - * of the AbstractSliderLaunchedService instance. - */ - public RegistryOperations startRegistryOperationsService() - throws BadConfigException { - - // push back the slider registry entry if needed - RegistryOperations registryWriterService = - createRegistryOperationsInstance(); - deployChildService(registryWriterService); - return registryWriterService; - } - - /** - * Create the registry operations instance. This is to allow - * subclasses to instantiate a subclass service - * @return an instance to match to the lifecycle of this service - */ - protected RegistryOperations createRegistryOperationsInstance() { - return RegistryOperationsFactory.createInstance("YarnRegistry", getConfig()); - } - - /** - * Utility method to require an argument to be set (non null, non-empty) - * @param argname argument name - * @param value value - * @throws BadCommandArgumentsException if the condition is not met - */ - protected static void requireArgumentSet(String argname, String value) - throws BadCommandArgumentsException { - require(isSet(value), "Required argument %s missing", argname ); - } - - /** - * Require a condition to hold; throw {@link BadCommandArgumentsException} if not. - * The exception text is the formatted message. - * @param condition condition - * @param message string to format - * @param args list of arguments to format. - * @throws BadCommandArgumentsException - */ - protected static void require(boolean condition, String message, - Object... args) - throws BadCommandArgumentsException { - if (!condition) { - throw new BadCommandArgumentsException(message, args); - } - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/EndOfServiceWaiter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/EndOfServiceWaiter.java deleted file mode 100644 index 40ceab8..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/EndOfServiceWaiter.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * 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.slider.server.services.utility; - -import org.apache.hadoop.service.Service; -import org.apache.hadoop.service.ServiceStateChangeListener; - -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * Wait for a service to stop. - * - * WARNING: the notification may come in as soon as the service enters - * the stopped state: it may take some time for the actual stop operation - * to complete. - */ -public class EndOfServiceWaiter implements ServiceStateChangeListener { - - private final AtomicBoolean finished = new AtomicBoolean(false); - private final String name; - private Service service; - - /** - * Wait for a service; use the service name as this instance's name - * @param service service - */ - public EndOfServiceWaiter(Service service) { - this(service.getName(), service); - } - - - /** - * Wait for a service - * @param name name for messages - * @param service service - */ - public EndOfServiceWaiter(String name, Service service) { - this.name = name; - this.service = service; - service.registerServiceListener(this); - } - - public synchronized void waitForServiceToStop(long timeout) throws - InterruptedException, TimeoutException { - service.waitForServiceToStop(timeout); - if (!finished.get()) { - wait(timeout); - if (!finished.get()) { - throw new TimeoutException(name - + " did not finish after " + timeout + - " milliseconds"); - } - } - } - - /** - * Wait for service state change callbacks; notify self if the service has - * now stopped - * @param service service - */ - @Override - public synchronized void stateChanged(Service service) { - if (service.isInState(Service.STATE.STOPPED)) { - finished.set(true); - notify(); - } - } - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/LaunchedWorkflowCompositeService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/LaunchedWorkflowCompositeService.java deleted file mode 100644 index bcd1969..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/LaunchedWorkflowCompositeService.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * 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.slider.server.services.utility; - -import com.google.common.base.Preconditions; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.Service; -import org.apache.slider.core.main.LauncherExitCodes; -import org.apache.slider.core.main.RunService; -import org.apache.slider.server.services.workflow.WorkflowCompositeService; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This is a workflow compositoe service which can be launched from the CLI - * ... catches the arguments and implements a stub runService operation. - */ -public class LaunchedWorkflowCompositeService extends WorkflowCompositeService - implements RunService { - private static final Logger log = LoggerFactory.getLogger( - LaunchedWorkflowCompositeService.class); - private String[] argv; - - public LaunchedWorkflowCompositeService(String name) { - super(name); - } - - public LaunchedWorkflowCompositeService(String name, Service... children) { - super(name, children); - } - - /** - * Implementation of set-ness, groovy definition of true/false for a string - * @param s - * @return true iff the string is non-null and non-empty - */ - protected static boolean isUnset(String s) { - return StringUtils.isEmpty(s); - } - - protected static boolean isSet(String s) { - return StringUtils.isNotEmpty(s); - } - - protected String[] getArgv() { - return argv; - } - - /** - * Pre-init argument binding - * @param config the initial configuration build up by the - * service launcher. - * @param args argument list list of arguments passed to the command line - * after any launcher-specific commands have been stripped. - * @return the configuration - * @throws Exception - */ - @Override - public Configuration bindArgs(Configuration config, String... args) throws - Exception { - this.argv = args; - if (log.isDebugEnabled()) { - log.debug("Binding {} Arguments:", args.length); - - StringBuilder builder = new StringBuilder(); - for (String arg : args) { - builder.append('"').append(arg).append("\" "); - } - log.debug(builder.toString()); - } - return config; - } - - @Override - public int runService() throws Throwable { - return LauncherExitCodes.EXIT_SUCCESS; - } - - @Override - public synchronized void addService(Service service) { - Preconditions.checkArgument(service != null, "null service argument"); - super.addService(service); - } - - /** - * Run a child service -initing and starting it if this - * service has already passed those parts of its own lifecycle - * @param service the service to start - */ - protected boolean deployChildService(Service service) { - service.init(getConfig()); - addService(service); - if (isInState(STATE.STARTED)) { - service.start(); - return true; - } - return false; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/WebAppService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/WebAppService.java deleted file mode 100644 index ebfcb99..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/WebAppService.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.slider.server.services.utility; - -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.yarn.webapp.WebApp; - -/** - * Contains a webapp reference and stops it in teardown if non-null - *

- * It does not start the application. - * Access to the field is not synchronized across threads; it is the - * responsibility of the caller. - */ -public class WebAppService extends AbstractService { - - private volatile T webApp; - - public WebAppService(String name) { - super(name); - } - - public WebAppService(String name, T app) { - super(name); - webApp = app; - } - - public T getWebApp() { - return webApp; - } - - public void setWebApp(T webApp) { - this.webApp = webApp; - } - - - @Override - protected void serviceStart() throws Exception { - - } - - /** - * Stop operation stops the webapp; sets the reference to null - * @throws Exception - */ - @Override - protected void serviceStop() throws Exception { - if (webApp != null) { - webApp.stop(); - webApp = null; - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ClosingService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ClosingService.java deleted file mode 100644 index 8b711aa..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ClosingService.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import org.apache.hadoop.service.AbstractService; - -import java.io.Closeable; -import java.io.IOException; - -/** - * Service that closes the closeable supplied during shutdown, if not null. - * - * As the Service interface itself extends Closeable, this service - * can be used to shut down other services if desired. - */ -public class ClosingService extends AbstractService { - - private C closeable; - - public ClosingService(String name) { - super(name); - } - - /** - * Construct an instance of the service - * @param name service name - * @param closeable closeable to close (may be null) - */ - public ClosingService(String name, - C closeable) { - super(name); - this.closeable = closeable; - } - - /** - * Construct an instance of the service, using the default name - * @param closeable closeable to close (may be null) - */ - public ClosingService(C closeable) { - this("ClosingService", closeable); - } - - - /** - * Get the closeable - * @return the closeable - */ - public synchronized C getCloseable() { - return closeable; - } - - /** - * Set or update the closeable. - * @param closeable - */ - public synchronized void setCloseable(C closeable) { - this.closeable = closeable; - } - - /** - * Stop routine will close the closeable -if not null - and set the - * reference to null afterwards - * This operation does raise any exception on the close, though it does - * record it - */ - @Override - protected void serviceStop() { - C target = getCloseable(); - if (target != null) { - try { - target.close(); - } catch (IOException ioe) { - noteFailure(ioe); - } - setCloseable(null); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java deleted file mode 100644 index 352be49..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java +++ /dev/null @@ -1,301 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import org.apache.hadoop.service.ServiceStateException; -import org.apache.slider.core.main.ServiceLaunchException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * Service wrapper for an external program that is launched and can/will terminate. - * This service is notified when the subprocess terminates, and stops itself - * and converts a non-zero exit code into a failure exception. - * - *

- * Key Features: - *

    - *
  1. The property {@link #executionTimeout} can be set to set a limit - * on the duration of a process
  2. - *
  3. Output is streamed to the output logger provided
  4. . - *
  5. The most recent lines of output are saved to a linked list
  6. . - *
  7. A synchronous callback, {@link LongLivedProcessLifecycleEvent}, is raised on the start - * and finish of a process.
  8. - *
- * - * Usage: - *

- * The service can be built in the constructor, {@link #ForkedProcessService(String, Map, List)}, - * or have its simple constructor used to instantiate the service, then the - * {@link #build(Map, List)} command used to define the environment variables - * and list of commands to execute. One of these two options MUST be exercised - * before calling the services's {@link #start()} method. - *

- * The forked process is executed in the service's {@link #serviceStart()} method; - * if still running when the service is stopped, {@link #serviceStop()} will - * attempt to stop it. - *

- * - * The service delegates process execution to {@link LongLivedProcess}, - * receiving callbacks via the {@link LongLivedProcessLifecycleEvent}. - * When the service receives a callback notifying that the process has completed, - * it calls its {@link #stop()} method. If the error code was non-zero, - * the service is logged as having failed. - */ -public class ForkedProcessService - extends WorkflowExecutorService - implements LongLivedProcessLifecycleEvent, Runnable { - - /** - * Log for the forked master process - */ - private static final Logger LOG = - LoggerFactory.getLogger(ForkedProcessService.class); - - private final AtomicBoolean processTerminated = new AtomicBoolean(false); - private boolean processStarted = false; - private LongLivedProcess process; - private int executionTimeout = -1; - private int timeoutCode = 1; - /** - log to log to; defaults to this service log - */ - private Logger processLog = LOG; - - /** - * Exit code set when the spawned process exits - */ - private AtomicInteger exitCode = new AtomicInteger(0); - - /** - * Create an instance of the service - * @param name a name - */ - public ForkedProcessService(String name) { - super(name); - } - - /** - * Create an instance of the service, set up the process - * @param name a name - * @param commandList list of commands is inserted on the front - * @param env environment variables above those generated by - * @throws IOException IO problems - */ - public ForkedProcessService(String name, - Map env, - List commandList) throws IOException { - super(name); - build(env, commandList); - } - - @Override //AbstractService - protected void serviceStart() throws Exception { - if (process == null) { - throw new ServiceStateException("Process not yet configured"); - } - //now spawn the process -expect updates via callbacks - process.start(); - } - - @Override //AbstractService - protected void serviceStop() throws Exception { - completed(); - stopForkedProcess(); - } - - private void stopForkedProcess() { - if (process != null) { - process.stop(); - } - } - - /** - * Set the process log. This may be null for "do not log" - * @param processLog process log - */ - public void setProcessLog(Logger processLog) { - this.processLog = processLog; - process.setProcessLog(processLog); - } - - /** - * Set the timeout by which time a process must have finished -or -1 for forever - * @param timeout timeout in milliseconds - */ - public void setTimeout(int timeout, int code) { - this.executionTimeout = timeout; - this.timeoutCode = code; - } - - /** - * Build the process to execute when the service is started - * @param commandList list of commands is inserted on the front - * @param env environment variables above those generated by - * @throws IOException IO problems - */ - public void build(Map env, - List commandList) - throws IOException { - assert process == null; - - process = new LongLivedProcess(getName(), processLog, commandList); - process.setLifecycleCallback(this); - //set the env variable mapping - process.putEnvMap(env); - } - - @Override // notification from executed process - public synchronized void onProcessStarted(LongLivedProcess process) { - LOG.debug("Process has started"); - processStarted = true; - if (executionTimeout > 0) { - setExecutor(ServiceThreadFactory.singleThreadExecutor(getName(), true)); - execute(this); - } - } - - @Override // notification from executed process - public void onProcessExited(LongLivedProcess process, - int uncorrected, - int code) { - try { - synchronized (this) { - completed(); - //note whether or not the service had already stopped - LOG.debug("Process has exited with exit code {}", code); - if (code != 0) { - reportFailure(code, getName() + " failed with code " + code); - } - } - } finally { - stop(); - } - } - - private void reportFailure(int code, String text) { - //error - ServiceLaunchException execEx = new ServiceLaunchException(code, text); - LOG.debug("Noting failure", execEx); - noteFailure(execEx); - } - - /** - * handle timeout response by escalating it to a failure - */ - @Override - public void run() { - try { - synchronized (processTerminated) { - if (!processTerminated.get()) { - processTerminated.wait(executionTimeout); - } - } - - } catch (InterruptedException e) { - //assume signalled; exit - } - //check the status; if the marker isn't true, bail - if (!processTerminated.getAndSet(true)) { - LOG.info("process timeout: reporting error code {}", timeoutCode); - - //timeout - if (isInState(STATE.STARTED)) { - //trigger a failure - stopForkedProcess(); - } - reportFailure(timeoutCode, getName() + ": timeout after " + executionTimeout - + " millis: exit code =" + timeoutCode); - } - } - - /** - * Note the process as having completed. - * The process marked as terminated - * -and anything synchronized on processTerminated - * is notified - */ - protected void completed() { - processTerminated.set(true); - synchronized (processTerminated) { - processTerminated.notify(); - } - } - - public boolean isProcessTerminated() { - return processTerminated.get(); - } - - public synchronized boolean isProcessStarted() { - return processStarted; - } - - /** - * Is a process running: between started and terminated - * @return true if the process is up. - */ - public synchronized boolean isProcessRunning() { - return processStarted && !isProcessTerminated(); - } - - - public Integer getExitCode() { - return process.getExitCode(); - } - - public int getExitCodeSignCorrected() { - Integer exitCode = process.getExitCodeSignCorrected(); - if (exitCode == null) return -1; - return exitCode; - } - - /** - * Get the recent output from the process, or [] if not defined - * @return a possibly empty list - */ - public List getRecentOutput() { - return process != null - ? process.getRecentOutput() - : new LinkedList(); - } - - /** - * Get the recent output from the process, or [] if not defined - * - * @param finalOutput flag to indicate "wait for the final output of the process" - * @param duration the duration, in ms, - * to wait for recent output to become non-empty - * @return a possibly empty list - */ - public List getRecentOutput(boolean finalOutput, int duration) { - if (process == null) { - return new LinkedList<>(); - } - return process.getRecentOutput(finalOutput, duration); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java deleted file mode 100644 index 90a8d40..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java +++ /dev/null @@ -1,599 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.io.IOUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.BufferedReader; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStreamReader; -import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.Collections; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * Execute a long-lived process. - * - *

- * Hadoop's {@link org.apache.hadoop.util.Shell} class assumes it is executing - * a short lived application; this class allows for the process to run for the - * life of the Java process that forked it. - * It is designed to be embedded inside a YARN service, though this is not - * the sole way that it can be used - *

- * Key Features: - *

    - *
  1. Output is streamed to the output logger provided
  2. . - *
  3. the input stream is closed as soon as the process starts.
  4. - *
  5. The most recent lines of output are saved to a linked list
  6. . - *
  7. A synchronous callback, {@link LongLivedProcessLifecycleEvent}, - * is raised on the start and finish of a process.
  8. - *
- * - */ -public class LongLivedProcess implements Runnable { - /** - * Limit on number of lines to retain in the "recent" line list:{@value} - */ - public static final int RECENT_LINE_LOG_LIMIT = 64; - - /** - * Const defining the time in millis between polling for new text. - */ - private static final int STREAM_READER_SLEEP_TIME = 200; - - /** - * limit on the length of a stream before it triggers an automatic newline. - */ - private static final int LINE_LENGTH = 256; - private final ProcessBuilder processBuilder; - private Process process; - private Integer exitCode = null; - private final String name; - private final ExecutorService processExecutor; - private final ExecutorService logExecutor; - - private ProcessStreamReader processStreamReader; - //list of recent lines, recorded for extraction into reports - private final List recentLines = new LinkedList<>(); - private int recentLineLimit = RECENT_LINE_LOG_LIMIT; - private LongLivedProcessLifecycleEvent lifecycleCallback; - private final AtomicBoolean finalOutputProcessed = new AtomicBoolean(false); - - /** - * Log supplied in the constructor for the spawned process -accessible - * to inner classes - */ - private Logger processLog; - - /** - * Class log -accessible to inner classes - */ - private static final Logger LOG = LoggerFactory.getLogger(LongLivedProcess.class); - - /** - * flag to indicate that the process is done - */ - private final AtomicBoolean finished = new AtomicBoolean(false); - - /** - * Create an instance - * @param name process name - * @param processLog log for output (or null) - * @param commands command list - */ - public LongLivedProcess(String name, - Logger processLog, - List commands) { - Preconditions.checkArgument(commands != null, "commands"); - - this.name = name; - this.processLog = processLog; - ServiceThreadFactory factory = new ServiceThreadFactory(name, true); - processExecutor = Executors.newSingleThreadExecutor(factory); - logExecutor = Executors.newSingleThreadExecutor(factory); - processBuilder = new ProcessBuilder(commands); - processBuilder.redirectErrorStream(false); - } - - /** - * Set the limit on recent lines to retain - * @param recentLineLimit size of rolling list of recent lines. - */ - public void setRecentLineLimit(int recentLineLimit) { - this.recentLineLimit = recentLineLimit; - } - - /** - * Set an optional application exit callback - * @param lifecycleCallback callback to notify on application exit - */ - public void setLifecycleCallback(LongLivedProcessLifecycleEvent lifecycleCallback) { - this.lifecycleCallback = lifecycleCallback; - } - - /** - * Add an entry to the environment - * @param envVar envVar -must not be null - * @param val value - */ - public void setEnv(String envVar, String val) { - Preconditions.checkArgument(envVar != null, "envVar"); - Preconditions.checkArgument(val != null, "val"); - processBuilder.environment().put(envVar, val); - } - - /** - * Bulk set the environment from a map. This does - * not replace the existing environment, just extend it/overwrite single - * entries. - * @param map map to add - */ - public void putEnvMap(Map map) { - for (Map.Entry entry : map.entrySet()) { - String val = entry.getValue(); - String key = entry.getKey(); - setEnv(key, val); - } - } - - /** - * Get the process environment - * @param variable environment variable - * @return the value or null if there is no match - */ - public String getEnv(String variable) { - return processBuilder.environment().get(variable); - } - - /** - * Set the process log. Ignored once the process starts - * @param processLog new log ... may be null - */ - public void setProcessLog(Logger processLog) { - this.processLog = processLog; - } - - /** - * Get the process reference - * @return the process -null if the process is not started - */ - public Process getProcess() { - return process; - } - - /** - * Get the process builder -this can be manipulated - * up to the start() operation. As there is no synchronization - * around it, it must only be used in the same thread setting up the commmand. - * @return the process builder - */ - public ProcessBuilder getProcessBuilder() { - return processBuilder; - } - - /** - * Get the command list - * @return the comands - */ - public List getCommands() { - return processBuilder.command(); - } - - public String getCommand() { - return getCommands().get(0); - } - - /** - * probe to see if the process is running - * @return true iff the process has been started and is not yet finished - */ - public boolean isRunning() { - return process != null && !finished.get(); - } - - /** - * Get the exit code: null until the process has finished - * @return the exit code or null - */ - public Integer getExitCode() { - return exitCode; - } - - /** - * Get the exit code sign corrected: null until the process has finished - * @return the exit code or null - */ - public Integer getExitCodeSignCorrected() { - Integer result; - if (exitCode != null) { - result = (exitCode << 24) >> 24; - } else { - result = null; - } - return result; - } - - /** - * Stop the process if it is running. - * This will trigger an application completion event with the given exit code - */ - public void stop() { - if (!isRunning()) { - return; - } - process.destroy(); - } - - /** - * Get a text description of the builder suitable for log output - * @return a multiline string - */ - protected String describeBuilder() { - StringBuilder buffer = new StringBuilder(); - for (String arg : processBuilder.command()) { - buffer.append('"').append(arg).append("\" "); - } - return buffer.toString(); - } - - /** - * Dump the environment to a string builder - * @param buffer the buffer to append to - */ - public void dumpEnv(StringBuilder buffer) { - buffer.append("\nEnvironment\n-----------"); - Map env = processBuilder.environment(); - Set keys = env.keySet(); - List sortedKeys = new ArrayList(keys); - Collections.sort(sortedKeys); - for (String key : sortedKeys) { - buffer.append(key).append("=").append(env.get(key)).append('\n'); - } - } - - /** - * Exec the process - * @return the process - * @throws IOException on aany failure to start the process - * @throws FileNotFoundException if the process could not be found - */ - private Process spawnChildProcess() throws IOException { - if (process != null) { - throw new IOException("Process already started"); - } - if (LOG.isDebugEnabled()) { - LOG.debug("Spawning process:\n " + describeBuilder()); - } - try { - process = processBuilder.start(); - } catch (IOException e) { - // on windows, upconvert DOS error 2 from ::CreateProcess() - // to its real meaning: FileNotFound - if (e.toString().contains("CreateProcess error=2")) { - FileNotFoundException fnfe = - new FileNotFoundException(e.toString()); - fnfe.initCause(e); - throw fnfe; - } else { - throw e; - } - } - return process; - } - - /** - * Entry point for waiting for the program to finish - */ - @Override // Runnable - public void run() { - Preconditions.checkNotNull(process, "null process"); - LOG.debug("Lifecycle callback thread running"); - //notify the callback that the process has started - if (lifecycleCallback != null) { - lifecycleCallback.onProcessStarted(this); - } - try { - //close stdin for the process - IOUtils.closeStream(process.getOutputStream()); - exitCode = process.waitFor(); - } catch (InterruptedException e) { - LOG.debug("Process wait interrupted -exiting thread", e); - } finally { - //here the process has finished - LOG.debug("process {} has finished", name); - //tell the logger it has to finish too - finished.set(true); - - // shut down the threads - logExecutor.shutdown(); - try { - logExecutor.awaitTermination(60, TimeUnit.SECONDS); - } catch (InterruptedException ignored) { - //ignored - } - - //now call the callback if it is set - if (lifecycleCallback != null) { - lifecycleCallback.onProcessExited(this, exitCode, - getExitCodeSignCorrected()); - } - } - } - - /** - * Spawn the application - * @throws IOException IO problems - */ - public void start() throws IOException { - - spawnChildProcess(); - processStreamReader = - new ProcessStreamReader(processLog, STREAM_READER_SLEEP_TIME); - logExecutor.submit(processStreamReader); - processExecutor.submit(this); - } - - /** - * Get the lines of recent output - * @return the last few lines of output; an empty list if there are none - * or the process is not actually running - */ - public synchronized List getRecentOutput() { - return new ArrayList(recentLines); - } - - /** - * @return whether lines of recent output are empty - */ - public synchronized boolean isRecentOutputEmpty() { - return recentLines.isEmpty(); - } - - /** - * Query to see if the final output has been processed - * @return - */ - public boolean isFinalOutputProcessed() { - return finalOutputProcessed.get(); - } - - /** - * Get the recent output from the process, or [] if not defined - * - * @param finalOutput flag to indicate "wait for the final output of the process" - * @param duration the duration, in ms, - * ro wait for recent output to become non-empty - * @return a possibly empty list - */ - public List getRecentOutput(boolean finalOutput, int duration) { - long start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start <= duration) { - boolean finishedOutput; - if (finalOutput) { - // final flag means block until all data is done - finishedOutput = isFinalOutputProcessed(); - } else { - // there is some output - finishedOutput = !isRecentOutputEmpty(); - } - if (finishedOutput) { - break; - } - try { - Thread.sleep(100); - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - break; - } - } - return getRecentOutput(); - } - - /** - * add the recent line to the list of recent lines; deleting - * an earlier on if the limit is reached. - * - * Implementation note: yes, a circular array would be more - * efficient, especially with some power of two as the modulo, - * but is it worth the complexity and risk of errors for - * something that is only called once per line of IO? - * @param line line to record - * @param isErrorStream is the line from the error stream - * @param logger logger to log to - null for no logging - */ - private synchronized void recordRecentLine(String line, - boolean isErrorStream, - Logger logger) { - if (line == null) { - return; - } - String entry = (isErrorStream ? "[ERR] " : "[OUT] ") + line; - recentLines.add(entry); - if (recentLines.size() > recentLineLimit) { - recentLines.remove(0); - } - if (logger != null) { - if (isErrorStream) { - logger.warn(line); - } else { - logger.info(line); - } - } - } - - /** - * Class to read data from the two process streams, and, when run in a thread - * to keep running until the done flag is set. - * Lines are fetched from stdout and stderr and logged at info and error - * respectively. - */ - - private class ProcessStreamReader implements Runnable { - private final Logger streamLog; - private final int sleepTime; - - /** - * Create an instance - * @param streamLog log -or null to disable logging (recent entries - * will still be retained) - * @param sleepTime time to sleep when stopping - */ - private ProcessStreamReader(Logger streamLog, int sleepTime) { - this.streamLog = streamLog; - this.sleepTime = sleepTime; - } - - /** - * Return a character if there is one, -1 if nothing is ready yet - * @param reader reader - * @return the value from the reader, or -1 if it is not ready - * @throws IOException IO problems - */ - private int readCharNonBlocking(BufferedReader reader) throws IOException { - if (reader.ready()) { - return reader.read(); - } else { - return -1; - } - } - - /** - * Read in a line, or, if the limit has been reached, the buffer - * so far - * @param reader source of data - * @param line line to build - * @param limit limit of line length - * @return true if the line can be printed - * @throws IOException IO trouble - */ - @SuppressWarnings("NestedAssignment") - private boolean readAnyLine(BufferedReader reader, - StringBuilder line, - int limit) - throws IOException { - int next; - while ((-1 != (next = readCharNonBlocking(reader)))) { - if (next != '\n') { - line.append((char) next); - limit--; - if (line.length() > limit) { - //enough has been read in to print it any - return true; - } - } else { - //line end return flag to say so - return true; - } - } - //here the end of the stream is hit, or the limit - return false; - } - - - @Override //Runnable - @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") - public void run() { - BufferedReader errReader = null; - BufferedReader outReader = null; - StringBuilder outLine = new StringBuilder(LINE_LENGTH); - StringBuilder errorLine = new StringBuilder(LINE_LENGTH); - try { - errReader = new BufferedReader( - new InputStreamReader(process.getErrorStream(), "UTF-8")); - outReader = new BufferedReader( - new InputStreamReader(process.getInputStream(), "UTF-8")); - while (!finished.get()) { - boolean processed = false; - if (readAnyLine(errReader, errorLine, LINE_LENGTH)) { - recordRecentLine(errorLine.toString(), true, streamLog); - errorLine.setLength(0); - processed = true; - } - if (readAnyLine(outReader, outLine, LINE_LENGTH)) { - recordRecentLine(outLine.toString(), false, streamLog); - outLine.setLength(0); - processed |= true; - } - if (!processed && !finished.get()) { - //nothing processed: wait a bit for data. - try { - Thread.sleep(sleepTime); - } catch (InterruptedException e) { - //ignore this, rely on the done flag - LOG.debug("Ignoring ", e); - } - } - } - // finished: cleanup - - //print the current error line then stream through the rest - recordFinalOutput(errReader, errorLine, true, streamLog); - //now do the info line - recordFinalOutput(outReader, outLine, false, streamLog); - - } catch (Exception ignored) { - LOG.warn("encountered {}", ignored, ignored); - //process connection has been torn down - } finally { - // close streams - IOUtils.closeStream(errReader); - IOUtils.closeStream(outReader); - //mark output as done - finalOutputProcessed.set(true); - } - } - - /** - * Record the final output of a process stream - * @param reader reader of output - * @param lineBuilder string builder into which line is built - * @param isErrorStream flag to indicate whether or not this is the - * is the line from the error stream - * @param logger logger to log to - * @throws IOException - */ - protected void recordFinalOutput(BufferedReader reader, - StringBuilder lineBuilder, boolean isErrorStream, Logger logger) throws - IOException { - String line = lineBuilder.toString(); - recordRecentLine(line, isErrorStream, logger); - line = reader.readLine(); - while (line != null) { - recordRecentLine(line, isErrorStream, logger); - line = reader.readLine(); - if (Thread.interrupted()) { - break; - } - } - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcessLifecycleEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcessLifecycleEvent.java deleted file mode 100644 index a13b508..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcessLifecycleEvent.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -/** - * Callback when a long-lived application exits - */ -public interface LongLivedProcessLifecycleEvent { - - /** - * Callback when a process is started - * @param process the process invoking the callback - */ - void onProcessStarted(LongLivedProcess process); - - /** - * Callback when a process has finished - * @param process the process invoking the callback - * @param exitCode exit code from the process - * @param signCorrectedCode the code- as sign corrected - */ - void onProcessExited(LongLivedProcess process, - int exitCode, - int signCorrectedCode); -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceParent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceParent.java deleted file mode 100644 index a123584..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceParent.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import org.apache.hadoop.service.Service; - -import java.util.List; - -/** - * Interface for accessing services that contain one or more child - * services. - */ -public interface ServiceParent extends Service { - - /** - * Add a child service. It must be in a consistent state with the - * service to which it is being added. - * @param service the service to add. - */ - void addService(Service service); - - /** - * Get an unmodifiable list of services - * @return a list of child services at the time of invocation - - * added services will not be picked up. - */ - List getServices(); -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingCallable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingCallable.java deleted file mode 100644 index 5ebf77c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingCallable.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.service.Service; - -import java.util.concurrent.Callable; - -/** - * A runnable which terminates its owner; it also catches any - * exception raised and can serve it back. - * - */ -public class ServiceTerminatingCallable implements Callable { - - private final Service owner; - private Exception exception; - /** - * This is the callback - */ - private final Callable callable; - - - /** - * Create an instance. If the owner is null, the owning service - * is not terminated. - * @param owner owning service -can be null - * @param callable callback. - */ - public ServiceTerminatingCallable(Service owner, - Callable callable) { - Preconditions.checkArgument(callable != null, "null callable"); - this.owner = owner; - this.callable = callable; - } - - - /** - * Get the owning service - * @return the service to receive notification when - * the runnable completes. - */ - public Service getOwner() { - return owner; - } - - /** - * Any exception raised by inner action's run. - * @return an exception or null. - */ - public Exception getException() { - return exception; - } - - /** - * Delegates the call to the callable supplied in the constructor, - * then calls the stop() operation on its owner. Any exception - * is caught, noted and rethrown - * @return the outcome of the delegated call operation - * @throws Exception if one was raised. - */ - @Override - public V call() throws Exception { - try { - return callable.call(); - } catch (Exception e) { - exception = e; - throw e; - } finally { - if (owner != null) { - owner.stop(); - } - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingRunnable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingRunnable.java deleted file mode 100644 index dc591df..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingRunnable.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.service.Service; - -/** - * A runnable which terminates its after running; it also catches any - * exception raised and can serve it back. - */ -public class ServiceTerminatingRunnable implements Runnable { - - private final Service owner; - private final Runnable action; - private Exception exception; - - /** - * Create an instance. - * @param owner owning service - * @param action action to execute before terminating the service - */ - public ServiceTerminatingRunnable(Service owner, Runnable action) { - Preconditions.checkArgument(owner != null, "null owner"); - Preconditions.checkArgument(action != null, "null action"); - this.owner = owner; - this.action = action; - } - - /** - * Get the owning service. - * @return the service to receive notification when - * the runnable completes. - */ - public Service getOwner() { - return owner; - } - - /** - * Any exception raised by inner action's run. - * @return an exception or null. - */ - public Exception getException() { - return exception; - } - - @Override - public void run() { - try { - action.run(); - } catch (Exception e) { - exception = e; - } - owner.stop(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceThreadFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceThreadFactory.java deleted file mode 100644 index 737197b..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceThreadFactory.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import com.google.common.base.Preconditions; - -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * A thread factory that creates threads (possibly daemon threads) - * using the name and naming policy supplied. - * The thread counter starts at 1, increments atomically, - * and is supplied as the second argument in the format string. - * - * A static method, {@link #singleThreadExecutor(String, boolean)}, - * exists to simplify the construction of an executor with a single well-named - * threads. - * - * Example - *
- *  ExecutorService exec = ServiceThreadFactory.newSingleThreadExecutor("live", true)
- * 
- */ -public class ServiceThreadFactory implements ThreadFactory { - - private static final AtomicInteger counter = new AtomicInteger(1); - - /** - * Default format for thread names: {@value}. - */ - public static final String DEFAULT_NAMING_FORMAT = "%s-%03d"; - private final String name; - private final boolean daemons; - private final String namingFormat; - - /** - * Create an instance - * @param name base thread name - * @param daemons flag to indicate the threads should be marked as daemons - * @param namingFormat format string to generate thread names from - */ - public ServiceThreadFactory(String name, - boolean daemons, - String namingFormat) { - Preconditions.checkArgument(name != null, "null name"); - Preconditions.checkArgument(namingFormat != null, "null naming format"); - this.name = name; - this.daemons = daemons; - this.namingFormat = namingFormat; - } - - /** - * Create an instance with the default naming format. - * @param name base thread name - * @param daemons flag to indicate the threads should be marked as daemons - */ - public ServiceThreadFactory(String name, - boolean daemons) { - this(name, daemons, DEFAULT_NAMING_FORMAT); - } - - @Override - public Thread newThread(Runnable r) { - Preconditions.checkArgument(r != null, "null runnable"); - String threadName = - String.format(namingFormat, name, counter.getAndIncrement()); - Thread thread = new Thread(r, threadName); - thread.setDaemon(daemons); - return thread; - } - - /** - * Create a single thread executor using this naming policy. - * @param name base thread name - * @param daemons flag to indicate the threads should be marked as daemons - * @return an executor - */ - public static ExecutorService singleThreadExecutor(String name, - boolean daemons) { - return Executors.newSingleThreadExecutor( - new ServiceThreadFactory(name, daemons)); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCallbackService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCallbackService.java deleted file mode 100644 index 65d14b7..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCallbackService.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import com.google.common.base.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.concurrent.Callable; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; - -/** - * A service that calls the supplied callback when it is started -after the - * given delay. - * - * It can be configured to stop itself after the callback has - * completed, marking any exception raised as the exception of this service. - * The notifications come in on a callback thread -a thread that is only - * started in this service's start() operation. - */ -public class WorkflowCallbackService extends - WorkflowScheduledExecutorService { - protected static final Logger LOG = - LoggerFactory.getLogger(WorkflowCallbackService.class); - - /** - * This is the callback. - */ - private final Callable callback; - private final int delay; - private final ServiceTerminatingCallable command; - - private ScheduledFuture scheduledFuture; - - /** - * Create an instance of the service - * @param name service name - * @param callback callback to invoke - * @param delay delay -or 0 for no delay - * @param terminate terminate this service after the callback? - */ - public WorkflowCallbackService(String name, - Callable callback, - int delay, - boolean terminate) { - super(name); - Preconditions.checkNotNull(callback, "Null callback argument"); - this.callback = callback; - this.delay = delay; - command = new ServiceTerminatingCallable( - terminate ? this : null, - callback); - } - - public ScheduledFuture getScheduledFuture() { - return scheduledFuture; - } - - @Override - protected void serviceStart() throws Exception { - LOG.debug("Notifying {} after a delay of {} millis", callback, delay); - ScheduledExecutorService executorService = - Executors.newSingleThreadScheduledExecutor( - new ServiceThreadFactory(getName(), true)); - setExecutor(executorService); - scheduledFuture = - executorService.schedule(command, delay, TimeUnit.MILLISECONDS); - } - - /** - * Stop the service. - * If there is any exception noted from any executed notification, - * note the exception in this class - * @throws Exception exception. - */ - @Override - protected void serviceStop() throws Exception { - super.serviceStop(); - // propagate any failure - if (getCallbackException() != null) { - throw getCallbackException(); - } - } - - /** - * Get the exception raised by a callback. Will always be null if the - * callback has not been executed; will only be non-null after any success. - * @return a callback - */ - public Exception getCallbackException() { - return command.getException(); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCompositeService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCompositeService.java deleted file mode 100644 index 9c653f3..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCompositeService.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.CompositeService; -import org.apache.hadoop.service.Service; -import org.apache.hadoop.service.ServiceStateChangeListener; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; - -/** - * An extended composite service which stops itself if any child service - * fails, or when all its children have successfully stopped without failure. - * - * Lifecycle - *
    - *
  1. If any child exits with a failure: this service stops, propagating - * the exception.
  2. - *
  3. When all child services has stopped, this service stops itself
  4. - *
- * - */ -public class WorkflowCompositeService extends CompositeService - implements ServiceParent, ServiceStateChangeListener { - - private static final Logger LOG = - LoggerFactory.getLogger(WorkflowCompositeService.class); - - /** - * Deadlock-avoiding overridden config for slider services; see SLIDER-1052 - */ - private volatile Configuration configuration; - - /** - * Construct an instance - * @param name name of this service instance - */ - public WorkflowCompositeService(String name) { - super(name); - } - - @Override - public Configuration getConfig() { - return configuration; - } - - @Override - protected void setConfig(Configuration conf) { - super.setConfig(conf); - configuration = conf; - } - - /** - * Construct an instance with the default name. - */ - public WorkflowCompositeService() { - this("WorkflowCompositeService"); - } - - /** - * Varargs constructor - * @param name name of this service instance - * @param children children - */ - public WorkflowCompositeService(String name, Service... children) { - this(name); - for (Service child : children) { - addService(child); - } - } - - /** - * Construct with a list of children - * @param name name of this service instance - * @param children children to add - */ - public WorkflowCompositeService(String name, List children) { - this(name); - for (Service child : children) { - addService(child); - } - } - - /** - * Add a service, and register it - * @param service the {@link Service} to be added. - * Important: do not add a service to a parent during your own serviceInit/start, - * in Hadoop 2.2; you will trigger a ConcurrentModificationException. - */ - @Override - public synchronized void addService(Service service) { - Preconditions.checkArgument(service != null, "null service argument"); - service.registerServiceListener(this); - super.addService(service); - } - - /** - * When this service is started, any service stopping with a failure - * exception is converted immediately into a failure of this service, - * storing the failure and stopping ourselves. - * @param child the service that has changed. - */ - @Override - public void stateChanged(Service child) { - //if that child stopped while we are running: - if (isInState(STATE.STARTED) && child.isInState(STATE.STOPPED)) { - // a child service has stopped - //did the child fail? if so: propagate - Throwable failureCause = child.getFailureCause(); - if (failureCause != null) { - LOG.info("Child service " + child + " failed", failureCause); - //failure. Convert to an exception - Exception e = (failureCause instanceof Exception) ? - (Exception) failureCause : new Exception(failureCause); - //flip ourselves into the failed state - noteFailure(e); - stop(); - } else { - LOG.info("Child service completed {}", child); - if (areAllChildrenStopped()) { - LOG.info("All children are halted: stopping"); - stop(); - } - } - } - } - - /** - * Probe to query if all children are stopped -simply - * by taking a snapshot of the child service list and enumerating - * their state. - * The state of the children may change during this operation -that will - * not get picked up. - * @return true if all the children are stopped. - */ - private boolean areAllChildrenStopped() { - List children = getServices(); - boolean stopped = true; - for (Service child : children) { - if (!child.isInState(STATE.STOPPED)) { - stopped = false; - break; - } - } - return stopped; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowExecutorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowExecutorService.java deleted file mode 100644 index 7409d32..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowExecutorService.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.service.AbstractService; - -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; - -/** - * A service that hosts an executor -when the service is stopped, - * {@link ExecutorService#shutdownNow()} is invoked. - */ -public class WorkflowExecutorService extends AbstractService { - - private E executor; - - /** - * Construct an instance with the given name -but - * no executor - * @param name service name - */ - public WorkflowExecutorService(String name) { - this(name, null); - } - - /** - * Construct an instance with the given name and executor - * @param name service name - * @param executor exectuor - */ - public WorkflowExecutorService(String name, - E executor) { - super(name); - this.executor = executor; - } - - /** - * Get the executor - * @return the executor - */ - public synchronized E getExecutor() { - return executor; - } - - /** - * Set the executor. Only valid if the current one is null - * @param executor executor - */ - public synchronized void setExecutor(E executor) { - Preconditions.checkState(this.executor == null, - "Executor already set"); - this.executor = executor; - } - - /** - * Execute the runnable with the executor (which - * must have been created already) - * @param runnable runnable to execute - */ - public void execute(Runnable runnable) { - getExecutor().execute(runnable); - } - - /** - * Submit a callable - * @param callable callable - * @param type of the final get - * @return a future to wait on - */ - public Future submit(Callable callable) { - return getExecutor().submit(callable); - } - - /** - * Stop the service: halt the executor. - * @throws Exception exception. - */ - @Override - protected void serviceStop() throws Exception { - stopExecutor(); - super.serviceStop(); - } - - /** - * Stop the executor if it is not null. - * This uses {@link ExecutorService#shutdownNow()} - * and so does not block until they have completed. - */ - protected synchronized void stopExecutor() { - if (executor != null) { - executor.shutdownNow(); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowRpcService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowRpcService.java deleted file mode 100644 index b71530f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowRpcService.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.ipc.Server; -import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.service.AbstractService; - -import java.net.InetSocketAddress; - -/** - * A YARN service that maps the start/stop lifecycle of an RPC server - * to the YARN service lifecycle. - */ -public class WorkflowRpcService extends AbstractService { - - /** RPC server*/ - private final Server server; - - /** - * Construct an instance - * @param name service name - * @param server service to stop - */ - public WorkflowRpcService(String name, Server server) { - super(name); - Preconditions.checkArgument(server != null, "Null server"); - this.server = server; - } - - /** - * Get the server - * @return the server - */ - public Server getServer() { - return server; - } - - /** - * Get the socket address of this server - * @return the address this server is listening on - */ - public InetSocketAddress getConnectAddress() { - return NetUtils.getConnectAddress(server); - } - - @Override - protected void serviceStart() throws Exception { - super.serviceStart(); - server.start(); - } - - @Override - protected void serviceStop() throws Exception { - if (server != null) { - server.stop(); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowScheduledExecutorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowScheduledExecutorService.java deleted file mode 100644 index e9f53ed..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowScheduledExecutorService.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import java.util.concurrent.ScheduledExecutorService; - -/** - * Scheduled executor or subclass thereof - * @param scheduled executor service type - */ -public class WorkflowScheduledExecutorService - extends WorkflowExecutorService { - - public WorkflowScheduledExecutorService(String name) { - super(name); - } - - public WorkflowScheduledExecutorService(String name, - E executor) { - super(name, executor); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowSequenceService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowSequenceService.java deleted file mode 100644 index 97f97e8..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowSequenceService.java +++ /dev/null @@ -1,306 +0,0 @@ -/** - * 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.slider.server.services.workflow; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.service.Service; -import org.apache.hadoop.service.ServiceStateChangeListener; -import org.apache.hadoop.service.ServiceStateException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -/** - * This resembles the YARN CompositeService, except that it - * starts one service after another - * - * Workflow - *
    - *
  1. When the WorkflowSequenceService instance is - * initialized, it only initializes itself.
  2. - * - *
  3. When the WorkflowSequenceService instance is - * started, it initializes then starts the first of its children. - * If there are no children, it immediately stops.
  4. - * - *
  5. When the active child stops, it did not fail, and the parent has not - * stopped -then the next service is initialized and started. If there is no - * remaining child the parent service stops.
  6. - * - *
  7. If the active child did fail, the parent service notes the exception - * and stops -effectively propagating up the failure. - *
  8. - *
- * - * New service instances MAY be added to a running instance -but no guarantees - * can be made as to whether or not they will be run. - */ - -public class WorkflowSequenceService extends AbstractService implements - ServiceParent, ServiceStateChangeListener { - - private static final Logger LOG = - LoggerFactory.getLogger(WorkflowSequenceService.class); - - /** - * list of services - */ - private final List serviceList = new ArrayList<>(); - - /** - * The currently active service. - * Volatile -may change & so should be read into a - * local variable before working with - */ - private volatile Service activeService; - - /** - the previous service -the last one that finished. - null if one did not finish yet - */ - private volatile Service previousService; - - private boolean stopIfNoChildServicesAtStartup = true; - - /** - * Construct an instance - * @param name service name - */ - public WorkflowSequenceService(String name) { - super(name); - } - - /** - * Construct an instance with the default name - */ - public WorkflowSequenceService() { - this("WorkflowSequenceService"); - } - - /** - * Create a service sequence with the given list of services - * @param name service name - * @param children initial sequence - */ - public WorkflowSequenceService(String name, Service... children) { - super(name); - for (Service service : children) { - addService(service); - } - } /** - * Create a service sequence with the given list of services - * @param name service name - * @param children initial sequence - */ - public WorkflowSequenceService(String name, List children) { - super(name); - for (Service service : children) { - addService(service); - } - } - - /** - * Get the current service -which may be null - * @return service running - */ - public Service getActiveService() { - return activeService; - } - - /** - * Get the previously active service - * @return the service last run, or null if there is none. - */ - public Service getPreviousService() { - return previousService; - } - - protected void setStopIfNoChildServicesAtStartup(boolean stopIfNoChildServicesAtStartup) { - this.stopIfNoChildServicesAtStartup = stopIfNoChildServicesAtStartup; - } - - /** - * When started - * @throws Exception - */ - @Override - protected void serviceStart() throws Exception { - if (!startNextService() && stopIfNoChildServicesAtStartup) { - //nothing to start -so stop - stop(); - } - } - - @Override - protected void serviceStop() throws Exception { - //stop current service. - //this triggers a callback that is caught and ignored - Service current = activeService; - previousService = current; - activeService = null; - if (current != null) { - current.stop(); - } - } - - /** - * Start the next service in the list. - * Return false if there are no more services to run, or this - * service has stopped - * @return true if a service was started - * @throws RuntimeException from any init or start failure - * @throws ServiceStateException if this call is made before - * the service is started - */ - public synchronized boolean startNextService() { - if (isInState(STATE.STOPPED)) { - //downgrade to a failed - LOG.debug("Not starting next service -{} is stopped", this); - return false; - } - if (!isInState(STATE.STARTED)) { - //reject attempts to start a service too early - throw new ServiceStateException( - "Cannot start a child service when not started"); - } - if (serviceList.isEmpty()) { - //nothing left to run - return false; - } - if (activeService != null && activeService.getFailureCause() != null) { - //did the last service fail? Is this caused by some premature callback? - LOG.debug("Not starting next service due to a failure of {}", - activeService); - return false; - } - //bear in mind that init & start can fail, which - //can trigger re-entrant calls into the state change listener. - //by setting the current service to null - //the start-next-service logic is skipped. - //now, what does that mean w.r.t exit states? - - activeService = null; - Service head = serviceList.remove(0); - - try { - head.init(getConfig()); - head.registerServiceListener(this); - head.start(); - } catch (RuntimeException e) { - noteFailure(e); - throw e; - } - //at this point the service must have explicitly started & not failed, - //else an exception would have been raised - activeService = head; - return true; - } - - /** - * State change event relays service stop events to - * {@link #onServiceCompleted(Service)}. Subclasses can - * extend that with extra logic - * @param service the service that has changed. - */ - @Override - public void stateChanged(Service service) { - // only react to the state change when it is the current service - // and it has entered the STOPPED state - if (service == activeService && service.isInState(STATE.STOPPED)) { - onServiceCompleted(service); - } - } - - /** - * handler for service completion: base class starts the next service - * @param service service that has completed - */ - protected synchronized void onServiceCompleted(Service service) { - LOG.info("Running service stopped: {}", service); - previousService = activeService; - - - //start the next service if we are not stopped ourselves - if (isInState(STATE.STARTED)) { - - //did the service fail? if so: propagate - Throwable failureCause = service.getFailureCause(); - if (failureCause != null) { - Exception e = (failureCause instanceof Exception) ? - (Exception) failureCause : new Exception(failureCause); - noteFailure(e); - stop(); - } - - //start the next service - boolean started; - try { - started = startNextService(); - } catch (Exception e) { - //something went wrong here - noteFailure(e); - started = false; - } - if (!started) { - //no start because list is empty - //stop and expect the notification to go upstream - stop(); - } - } else { - //not started, so just note that the current service - //has gone away - activeService = null; - } - } - - /** - * Add the passed {@link Service} to the list of services managed by this - * {@link WorkflowSequenceService} - * @param service the {@link Service} to be added - */ - @Override - public synchronized void addService(Service service) { - Preconditions.checkArgument(service != null, "null service argument"); - LOG.debug("Adding service {} ", service.getName()); - synchronized (serviceList) { - serviceList.add(service); - } - } - - /** - * Get an unmodifiable list of services - * @return a list of child services at the time of invocation - - * added services will not be picked up. - */ - @Override //Parent - public synchronized List getServices() { - return Collections.unmodifiableList(serviceList); - } - - @Override // Object - public synchronized String toString() { - return super.toString() + "; current service " + activeService - + "; queued service count=" + serviceList.size(); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/package-info.java deleted file mode 100644 index 36d059a..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/package-info.java +++ /dev/null @@ -1,172 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -/** - -

- This package contains classes which can be aggregated to build up - complex workflows of services: sequences of operations, callbacks - and composite services with a shared lifespan. -

- -

- Core concepts: -

- - -

-The Workflow Services are set of Hadoop YARN services, all implementing -the {@link org.apache.hadoop.service.Service} API. -They are designed to be aggregated, to be composed to produce larger -composite services which than perform ordered operations, notify other services -when work has completed, and to propagate failure up the service hierarchy. -

-

-Service instances may a limited lifespan, and may self-terminate when -they consider it appropriate.

-

-Workflow Services that have children implement the -{@link org.apache.slider.server.services.workflow.ServiceParent} -class, which provides (thread-safe) access to the children -allowing new children -to be added, and existing children to be ennumerated. The implement policies -on how to react to the termination of children -so can sequence operations -which terminate themselves when complete. -

- -

-Workflow Services may be subclassed to extend their behavior, or to use them -in specific applications. Just as the standard -{@link org.apache.hadoop.service.CompositeService} -is often subclassed to aggregate child services, the -{@link org.apache.slider.server.services.workflow.WorkflowCompositeService} -can be used instead -adding the feature that failing services trigger automatic -parent shutdown. If that is the desired operational mode of a class, -swapping the composite service implementation may be sufficient to adopt it. -

- - -

How do the workflow services differ from the standard YARN services?

- -

- - There is exactly one standard YARN service for managing children, the - {@link org.apache.hadoop.service.CompositeService}. -

- The {@link org.apache.slider.server.services.workflow.WorkflowCompositeService} - shares the same model of "child services, all inited and started together". - Where it differs is that if any child service stops -either due to a failure - or to an action which invokes that service's - {@link org.apache.hadoop.service.Service#stop()} method. -

-

- -In contrast, the original CompositeService class starts its children -in its{@link org.apache.hadoop.service.Service#start()} method, but does not -listen or react to any child service halting. As a result, changes in child -state are not automatically detected or propagated, other than failures in -the actual init() and start() methods. -

- -

-If a child service runs until completed -that is it will not be stopped until -instructed to do so, and if it is only the parent service that attempts to -stop the child, then this difference is unimportant. -

-

- -However, if any service that depends upon all it child services running - -and if those child services are written so as to stop when they fail, using -the WorkflowCompositeService as a base class will enable the -parent service to be automatically notified of a child stopping. - -

-

-The {@link org.apache.slider.server.services.workflow.WorkflowSequenceService} -resembles the composite service in API, but its workflow is different. It -initializes and starts its children one-by-one, only starting the second after -the first one succeeds, the third after the second, etc. If any service in -the sequence fails, the parent WorkflowSequenceService stops, -reporting the same exception. -

- -

-The {@link org.apache.slider.server.services.workflow.ForkedProcessService}: -Executes a process when started, and binds to the life of that process. When the -process terminates, so does the service -and vice versa. This service enables -external processes to be executed as part of a sequence of operations -or, -using the {@link org.apache.slider.server.services.workflow.WorkflowCompositeService} -in parallel with other services, terminating the process when the other services -stop -and vice versa. -

- -

-The {@link org.apache.slider.server.services.workflow.WorkflowCallbackService} -executes a {@link java.util.concurrent.Callable} callback a specified delay -after the service is started, then potentially terminates itself. -This is useful for callbacks when a workflow reaches a specific point --or simply for executing arbitrary code in the workflow. - -

- - -

-Other Workflow Services -

- -There are some minor services that have proven useful within aggregate workflows, -and simply in applications which are built from composite YARN services. - -
    -
  • {@link org.apache.slider.server.services.workflow.WorkflowRpcService }: - Maintains a reference to an RPC {@link org.apache.hadoop.ipc.Server} instance. - When the service is started, so is the RPC server. Similarly, when the service - is stopped, so is the RPC server instance. -
  • - -
  • {@link org.apache.slider.server.services.workflow.ClosingService}: Closes - an instance of {@link java.io.Closeable} when the service is stopped. This - is purely a housekeeping class. -
  • - -
- - Lower-level classes -
    -
  • {@link org.apache.slider.server.services.workflow.ServiceTerminatingRunnable }: - A {@link java.lang.Runnable} which runs the runnable supplied in its constructor - then signals its owning service to stop once that runnable is completed. - Any exception raised in the run is stored. -
  • -
  • {@link org.apache.slider.server.services.workflow.WorkflowExecutorService}: - A base class for services that wish to have a {@link java.util.concurrent.ExecutorService} - with a lifespan mapped to that of a service. When the service is stopped, the - {@link java.util.concurrent.ExecutorService#shutdownNow()} method is called to - attempt to shut down all running tasks. -
  • -
  • {@link org.apache.slider.server.services.workflow.ServiceThreadFactory}: - This is a simple {@link java.util.concurrent.ThreadFactory} which generates - meaningful thread names. It can be used as a parameter to constructors of - {@link java.util.concurrent.ExecutorService} instances, to ensure that - log information can tie back text to the related services
  • -
- - - - */ diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/util/RestApiConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/util/RestApiConstants.java deleted file mode 100644 index daaf0e9..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/util/RestApiConstants.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.slider.util; - -public interface RestApiConstants { - String CONTEXT_ROOT = "/services/v1"; - String APPLICATIONS_API_RESOURCE_PATH = "/applications"; - String CONTAINERS_API_RESOURCE_PATH = "/containers"; - String SLIDER_APPMASTER_COMPONENT_NAME = "slider-appmaster"; - String SLIDER_CONFIG_SCHEMA = "http://example.org/specification/v2.0.0"; - String METAINFO_SCHEMA_VERSION = "2.1"; - String COMPONENT_TYPE_YARN_DOCKER = "yarn_docker"; - - String DEFAULT_START_CMD = "/bootstrap/privileged-centos6-sshd"; - String DEFAULT_COMPONENT_NAME = "default"; - String DEFAULT_IMAGE = "centos:centos6"; - String DEFAULT_NETWORK = "bridge"; - String DEFAULT_COMMAND_PATH = "/usr/bin/docker"; - String DEFAULT_USE_NETWORK_SCRIPT = "yes"; - - String PLACEHOLDER_APP_NAME = "${APP_NAME}"; - String PLACEHOLDER_APP_COMPONENT_NAME = "${APP_COMPONENT_NAME}"; - String PLACEHOLDER_COMPONENT_ID = "${COMPONENT_ID}"; - - String PROPERTY_REST_SERVICE_HOST = "REST_SERVICE_HOST"; - String PROPERTY_REST_SERVICE_PORT = "REST_SERVICE_PORT"; - String PROPERTY_APP_LIFETIME = "docker.lifetime"; - String PROPERTY_APP_RUNAS_USER = "APP_RUNAS_USER"; - Long DEFAULT_UNLIMITED_LIFETIME = -1l; - - Integer HTTP_STATUS_CODE_ACCEPTED = 202; - String ARTIFACT_TYPE_SLIDER_ZIP = "slider-zip"; - - Integer GET_APPLICATIONS_THREAD_POOL_SIZE = 200; - - String PROPERTY_PYTHON_PATH = "python.path"; - String PROPERTY_DNS_DEPENDENCY = "site.global.dns.dependency"; - - String COMMAND_ORDER_SUFFIX_START = "-START"; - String COMMAND_ORDER_SUFFIX_STARTED = "-RUNNING_BUT_UNREADY"; - String EXPORT_GROUP_NAME = "QuickLinks"; - - Integer ERROR_CODE_APP_DOES_NOT_EXIST = 404001; - Integer ERROR_CODE_APP_IS_NOT_RUNNING = 404002; - Integer ERROR_CODE_APP_SUBMITTED_BUT_NOT_RUNNING_YET = 404003; - Integer ERROR_CODE_APP_NAME_INVALID = 404004; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterMessages.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterMessages.proto deleted file mode 100644 index 691f861..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterMessages.proto +++ /dev/null @@ -1,392 +0,0 @@ -/** - * 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. - */ - -option java_package = "org.apache.slider.api.proto"; -option java_outer_classname = "Messages"; -option java_generic_services = true; -option java_generate_equals_and_hash = true; -package org.apache.slider.api; - -//import "Security.proto"; - -/* - Look at SliderClusterProtocol.proto to see how to build this -*/ - -message RoleInstanceState { - required string name = 1; - optional string role = 2; - required uint32 state = 4; - required uint32 exitCode = 5; - optional string command = 6; - optional string diagnostics = 7; - repeated string output = 8; - repeated string environment = 9; - required uint32 roleId = 10; - required bool released = 11; - required int64 createTime = 12; - required int64 startTime = 13; - required string host = 14; - required string hostURL = 15; - optional string appVersion = 16; -} - -/** - * stop the cluster - */ -message StopClusterRequestProto { - /** - message to include - */ - required string message = 1; -} - -/** - * stop the cluster - */ -message StopClusterResponseProto { -} - -/** - * upgrade the containers - */ -message UpgradeContainersRequestProto { - /** - message to include - */ - required string message = 1; - repeated string container = 2; - repeated string component = 3; -} - -/** - * upgrade the containers - */ -message UpgradeContainersResponseProto { -} - -message FlexComponentsRequestProto { - repeated ComponentCountProto components = 1; -} - -message ComponentCountProto { - optional string name = 1; - optional int64 numberOfContainers = 2; -} - -message FlexComponentsResponseProto { -} - -/** - * void request - */ -message GetJSONClusterStatusRequestProto { -} - -/** - * response - */ -message GetJSONClusterStatusResponseProto { - required string clusterSpec = 1; -} - -/** - * list the nodes in a role - */ -message ListNodeUUIDsByRoleRequestProto { - required string role = 1; -} - -/** - * list the nodes in a role - */ -message ListNodeUUIDsByRoleResponseProto { - repeated string uuid = 1 ; -} - -/** - * get a node - */ -message GetNodeRequestProto { - required string uuid = 1; -} - - -/** - * response on a node - */ -message GetNodeResponseProto { - required RoleInstanceState clusterNode = 1 ; -} - -/** - * list the nodes for the UUDs - */ -message GetClusterNodesRequestProto { - repeated string uuid = 1 ; -} - -/** - * list the nodes in a role - */ -message GetClusterNodesResponseProto { - repeated RoleInstanceState clusterNode = 1 ; -} - -/** - * Echo - */ -message EchoRequestProto { - required string text = 1; -} - -/** - * Echo reply - */ -message EchoResponseProto { - required string text = 1; -} - - -/** - * Kill a container - */ -message KillContainerRequestProto { - required string id = 1; -} - -/** - * Kill reply - */ -message KillContainerResponseProto { - required bool success = 1; -} - -/** - * AM suicide - */ -message AMSuicideRequestProto { - required string text = 1; - required int32 signal = 2; - required int32 delay = 3; -} - -/** - * AM suicide reply. For this to be returned implies - * a failure of the AM to kill itself - */ -message AMSuicideResponseProto { - -} - - -/** - * Ask for the instance definition details - */ -message GetInstanceDefinitionRequestProto { - -} - -/** - * Get the definition back as three separate JSON strings - */ -message GetInstanceDefinitionResponseProto { - required string internal = 1; - required string resources = 2; - required string application = 3; -} - - - /* ************************************************************************ - - REST model and operations. - Below here the operations and payloads designed to mimic - the REST API. That API is now the source of those - specificatations; this is simply a derivative. - - **************************************************************************/ - -/** - * See org.apache.slider.api.types.ApplicationLivenessInformation - */ -message ApplicationLivenessInformationProto { - optional bool allRequestsSatisfied = 1; - optional int32 requestsOutstanding = 2; -} - -/* - * see org.apache.slider.api.types.ComponentInformation - */ -message ComponentInformationProto { - optional string name = 1; - optional int32 priority = 2; - optional int32 desired = 3; - optional int32 actual = 4; - optional int32 releasing = 5; - optional int32 requested = 6; - optional int32 failed = 7; - optional int32 started = 8; - optional int32 startFailed = 9; - optional int32 completed = 10; - optional int32 totalRequested = 11; - optional string failureMessage =12; - optional int32 placementPolicy =13; - repeated string containers = 14; - optional int32 failedRecently = 15; - optional int32 nodeFailed = 16; - optional int32 preempted = 17; - optional int32 pendingAntiAffineRequestCount = 18; - optional bool isAARequestOutstanding = 19; -} - -/* - * see org.apache.slider.api.types.ContainerInformation - */ -message ContainerInformationProto { - optional string containerId = 1; - optional string component = 2; - optional bool released = 3; - optional int32 state = 4; - optional int32 exitCode = 5; - optional string diagnostics = 6; - optional int64 createTime = 7; - optional int64 startTime = 8; - repeated string output = 9; - optional string host = 10; - optional string hostURL = 11; - optional string placement = 12; - optional string appVersion = 13; -} - - -/* - * see org.apache.slider.api.types.PingInformation - */ -message PingInformationProto { - optional string text = 1; - optional string verb = 2; - optional string body = 3; - optional int64 time = 4; -} - -message NodeEntryInformationProto { - required int32 priority = 1; - required int32 requested = 2; - required int32 starting = 3; - required int32 startFailed = 4; - required int32 failed = 5; - required int32 failedRecently= 6; - required int32 preempted = 7; - required int32 live = 8; - required int32 releasing = 9; - required int64 lastUsed = 10; - required string name = 11; -} - -message NodeInformationProto { - required string hostname = 1; - required string state = 2; - required string httpAddress = 3; - required string rackName = 4; - required string labels = 5; - required string healthReport= 6; - required int64 lastUpdated = 7; - repeated NodeEntryInformationProto entries = 8; -} - -message GetModelRequestProto { -} - -message GetModelDesiredRequestProto { -} - -message GetModelDesiredAppconfRequestProto { -} - -message GetModelDesiredResourcesRequestProto { -} - -message GetModelResolvedAppconfRequestProto { -} - -message GetModelResolvedResourcesRequestProto { -} - -message GetModelLiveResourcesRequestProto { -} - -message GetLiveContainersRequestProto { -} - -message GetLiveContainersResponseProto { - repeated string names = 1; - repeated ContainerInformationProto containers = 2; -} - -message GetLiveContainerRequestProto { - required string containerId = 1; -} - - -message GetLiveComponentsRequestProto { -} - -message GetLiveComponentsResponseProto { - - repeated string names = 1; - repeated ComponentInformationProto components = 2; -} - -message GetLiveComponentRequestProto { - required string name = 1; -} - -message GetApplicationLivenessRequestProto { -} - -message EmptyPayloadProto { -} - -/** - Generic JSON, often containing data structures serialized as a string -*/ -message WrappedJsonProto { - required string json = 1; -} - -message GetCertificateStoreRequestProto { - optional string hostname = 1; - required string requesterId = 2; - required string password = 3; - required string type = 4; -} - -message GetCertificateStoreResponseProto { - required bytes store = 1; -} - -message GetLiveNodesRequestProto { -} - -message GetLiveNodesResponseProto { - repeated NodeInformationProto nodes = 1; -} - -message GetLiveNodeRequestProto { - required string name = 1; -} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterProtocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterProtocol.proto deleted file mode 100644 index 776ce28..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterProtocol.proto +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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. - */ - -option java_package = "org.apache.slider.api.proto"; -option java_outer_classname = "SliderClusterAPI"; -option java_generic_services = true; -option java_generate_equals_and_hash = true; -package org.apache.slider.api; - -/* - -Compiling - -Maven: How to do it as part of the build - mvn install -DskipTests -Pcompile-protobuf - -How to do it so as to get error messages - -protoc --java_out=src/main/java \ - -Isrc/main/proto src/main/proto/SliderClusterMessages.proto \ - src/main/proto/SliderClusterProtocol.proto - -Once happy: commit the changes - -*/ - -//import "Security.proto"; -import "SliderClusterMessages.proto"; - - -/** - * Protocol used from between Slider Client and AM - */ -service SliderClusterProtocolPB { - - /** - * Stop the cluster - */ - - rpc stopCluster(StopClusterRequestProto) - returns(StopClusterResponseProto); - - /** - * Upgrade containers - */ - rpc upgradeContainers(UpgradeContainersRequestProto) - returns(UpgradeContainersResponseProto); - - rpc flexComponents(FlexComponentsRequestProto) returns (FlexComponentsResponseProto); - - /** - * Get the current cluster status - */ - rpc getJSONClusterStatus(GetJSONClusterStatusRequestProto) - returns(GetJSONClusterStatusResponseProto); - - /** - * List all running nodes in a role - */ - rpc listNodeUUIDsByRole(ListNodeUUIDsByRoleRequestProto) - returns(ListNodeUUIDsByRoleResponseProto); - - /** - * Get the details on a node - */ - rpc getNode(GetNodeRequestProto) - returns(GetNodeResponseProto); - - /** - * Get the - * details on a list of nodes. - * Unknown nodes are not returned - * Important: the order of the results are undefined - */ - rpc getClusterNodes(GetClusterNodesRequestProto) - returns(GetClusterNodesResponseProto); - - /** - * echo some text - */ - rpc echo(EchoRequestProto) - returns(EchoResponseProto); - - /** - * kill a container - */ - rpc killContainer(KillContainerRequestProto) - returns(KillContainerResponseProto); - - /** - * kill the AM - */ - rpc amSuicide(AMSuicideRequestProto) - returns(AMSuicideResponseProto); - - /* ************************************************************************ - - REST model and operations. - Below here the operations and payloads designed to mimic - the REST API. That API is now the source of those - specificatations; this is simply a derivative. - - **************************************************************************/ - - rpc getLivenessInformation(GetApplicationLivenessRequestProto) - returns(ApplicationLivenessInformationProto); - - rpc getLiveContainers(GetLiveContainersRequestProto) - returns(GetLiveContainersResponseProto); - - rpc getLiveContainer(GetLiveContainerRequestProto) - returns(ContainerInformationProto); - - rpc getLiveComponents(GetLiveComponentsRequestProto) - returns(GetLiveComponentsResponseProto); - - rpc getLiveComponent(GetLiveComponentRequestProto) - returns(ComponentInformationProto); - - rpc getLiveNodes(GetLiveNodesRequestProto) - returns(GetLiveNodesResponseProto); - - rpc getLiveNode(GetLiveNodeRequestProto) - returns(NodeInformationProto); -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo deleted file mode 100644 index 9e67c15..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo +++ /dev/null @@ -1,15 +0,0 @@ -# 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. -org.apache.slider.server.appmaster.rpc.SliderRPCSecurityInfo diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/log4j.properties deleted file mode 100644 index 65a7ad0..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/log4j.properties +++ /dev/null @@ -1,52 +0,0 @@ -# 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 a log4j config for slider - -log4j.rootLogger=INFO,stdout -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} [%t] %-5p %c{3} (%F:%M(%L)) - %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 - - -#at debug this provides details on what is going on -log4j.logger.org.apache.slider=DEBUG -#log4j.logger.org.apache.slider.exec.RunLongLivedApp=ERROR - -log4j.logger.org.apache.hadoop.security=DEBUG -log4j.logger.org.apache.hadoop.yarn.service.launcher=DEBUG -log4j.logger.org.apache.hadoop.yarn.service=DEBUG -log4j.logger.org.apache.hadoop.yarn.client=DEBUG -#crank back on some noise -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.hadoop.yarn.client.RMProxy=WARN - -# for test runs we don't care about native code -log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR -# HDFS is noise on tets -log4j.logger.org.apache.hadoop.hdfs.server.datanode=WARN -log4j.logger.org.apache.hadoop.hdfs.server.namenode=WARN -log4j.logger.org.apache.hadoop.hdfs.server.blockmanagement=WARN -log4j.logger.org.apache.hadoop.hdfs=WARN - -log4j.logger.org.apache.zookeeper=WARN diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/agent.txt b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/agent.txt deleted file mode 100644 index 79c1972..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/agent.txt +++ /dev/null @@ -1,19 +0,0 @@ -/* - * 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 conf directory for the python agent \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command.json deleted file mode 100644 index 197a046..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command.json +++ /dev/null @@ -1,168 +0,0 @@ -{ - "roleCommand": "START", - "clusterName": "c1", - "hostname": "c6402.ambari.apache.org", - "hostLevelParams": { - "java_home": "/usr/jdk64/jdk1.7.0_45" - }, - "commandType": "EXECUTION_COMMAND", - "roleParams": {}, - "serviceName": "HBASE", - "role": "HBASE_MASTER", - "commandParams": {}, - "taskId": 24, - "public_hostname": "c6402.ambari.apache.org", - "configurations": { - "hbase-log4j": { - "log4j.threshold": "ALL", - "log4j.rootLogger": "${hbase.root.logger}", - "log4j.logger.org.apache.zookeeper": "INFO", - "log4j.logger.org.apache.hadoop.hbase": "DEBUG", - "log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher": "INFO", - "log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil": "INFO", - "log4j.category.SecurityLogger": "${hbase.security.logger}", - "log4j.appender.console": "org.apache.log4j.ConsoleAppender", - "log4j.appender.console.target": "System.err", - "log4j.appender.console.layout": "org.apache.log4j.PatternLayout", - "log4j.appender.console.layout.ConversionPattern": "%d{ISO8601} %-5p [%t] %c{2}: %m%n", - "log4j.appender.RFAS": "org.apache.log4j.RollingFileAppender", - "log4j.appender.RFAS.layout": "org.apache.log4j.PatternLayout", - "log4j.appender.RFAS.layout.ConversionPattern": "%d{ISO8601} %p %c: %m%n", - "log4j.appender.RFAS.MaxFileSize": "${hbase.security.log.maxfilesize}", - "log4j.appender.RFAS.MaxBackupIndex": "${hbase.security.log.maxbackupindex}", - "log4j.appender.RFAS.File": "${hbase.log.dir}/${hbase.security.log.file}", - "log4j.appender.RFA": "org.apache.log4j.RollingFileAppender", - "log4j.appender.RFA.layout": "org.apache.log4j.PatternLayout", - "log4j.appender.RFA.layout.ConversionPattern": "%d{ISO8601} %-5p [%t] %c{2}: %m%n", - "log4j.appender.RFA.MaxFileSize": "${hbase.log.maxfilesize}", - "log4j.appender.RFA.MaxBackupIndex": "${hbase.log.maxbackupindex}", - "log4j.appender.RFA.File": "${hbase.log.dir}/${hbase.log.file}", - "log4j.appender.NullAppender": "org.apache.log4j.varia.NullAppender", - "log4j.appender.DRFA": "org.apache.log4j.DailyRollingFileAppender", - "log4j.appender.DRFA.layout": "org.apache.log4j.PatternLayout", - "log4j.appender.DRFA.layout.ConversionPattern": "%d{ISO8601} %-5p [%t] %c{2}: %m%n", - "log4j.appender.DRFA.File": "${hbase.log.dir}/${hbase.log.file}", - "log4j.appender.DRFA.DatePattern": ".yyyy-MM-dd", - "log4j.additivity.SecurityLogger": "false", - "hbase.security.logger": "INFO,console", - "hbase.security.log.maxfilesize": "256MB", - "hbase.security.log.maxbackupindex": "20", - "hbase.security.log.file": "SecurityAuth.audit", - "hbase.root.logger": "INFO,console", - "hbase.log.maxfilesize": "256MB", - "hbase.log.maxbackupindex": "20", - "hbase.log.file": "hbase.log", - "hbase.log.dir": "." - }, - "global": { - "hbase_root": "/share/hbase/hbase-0.96.1-hadoop2", - "hbase_pid_dir": "/var/run/hbase", - "proxyuser_group": "users", - "syncLimit": "5", - "hbase_regionserver_heapsize": "1024m", - "rca_enabled": "false", - "tickTime": "2000", - "hbase_master_heapsize": "1024m", - "initLimit": "10", - "user_group": "hadoop", - "hbase_user": "hbase", - "hbase_log_dir": "/var/log/hbase" - }, - "hdfs-site": { - "dfs.namenode.checkpoint.period": "21600", - "dfs.namenode.avoid.write.stale.datanode": "true", - "dfs.namenode.checkpoint.txns": "1000000", - "dfs.block.access.token.enable": "true", - "dfs.support.append": "true", - "dfs.datanode.address": "0.0.0.0:${ambari.dfs.datanode.port}", - "dfs.cluster.administrators": " hdfs", - "dfs.replication": "3", - "ambari.dfs.datanode.http.port": "50075", - "dfs.datanode.balance.bandwidthPerSec": "6250000", - "dfs.namenode.safemode.threshold-pct": "1.0f", - "dfs.namenode.checkpoint.edits.dir": "${dfs.namenode.checkpoint.dir}", - "dfs.permissions.enabled": "true", - "dfs.client.read.shortcircuit": "true", - "dfs.namenode.https-address": "c6402.ambari.apache.org:50470", - "dfs.journalnode.edits.dir": "/grid/0/hdfs/journal", - "dfs.blocksize": "134217728", - "dfs.datanode.max.transfer.threads": "1024", - "dfs.datanode.du.reserved": "1073741824", - "dfs.webhdfs.enabled": "true", - "dfs.namenode.handler.count": "100", - "dfs.namenode.checkpoint.dir": "/hadoop/hdfs/namesecondary", - "fs.permissions.umask-mode": "022", - "dfs.datanode.http.address": "0.0.0.0:${ambari.dfs.datanode.http.port}", - "dfs.datanode.ipc.address": "0.0.0.0:8010", - "dfs.datanode.data.dir": "/hadoop/hdfs/data", - "dfs.namenode.http-address": "c6402.ambari.apache.org:50070", - "dfs.blockreport.initialDelay": "120", - "dfs.datanode.failed.volumes.tolerated": "0", - "dfs.namenode.accesstime.precision": "0", - "ambari.dfs.datanode.port": "50010", - "dfs.namenode.avoid.read.stale.datanode": "true", - "dfs.namenode.secondary.http-address": "c6402.ambari.apache.org:50090", - "dfs.namenode.stale.datanode.interval": "30000", - "dfs.heartbeat.interval": "3", - "dfs.client.read.shortcircuit.streams.cache.size": "4096", - "dfs.permissions.superusergroup": "hdfs", - "dfs.https.port": "50470", - "dfs.journalnode.http-address": "0.0.0.0:8480", - "dfs.domain.socket.path": "/var/lib/hadoop-hdfs/dn_socket", - "dfs.namenode.write.stale.datanode.ratio": "1.0f", - "dfs.hosts.exclude": "/etc/hadoop/conf/dfs.exclude", - "dfs.datanode.data.dir.perm": "750", - "dfs.namenode.name.dir.restore": "true", - "dfs.replication.max": "50", - "dfs.namenode.name.dir": "/hadoop/hdfs/namenode" - }, - "hbase-site": { - "hbase.hstore.flush.retries.number": "120", - "hbase.client.keyvalue.maxsize": "10485760", - "hbase.hstore.compactionThreshold": "3", - "hbase.rootdir": "hdfs://c6402.ambari.apache.org:8020/apps/hbase/data", - "hbase.stagingdir": "hdfs://c6402.ambari.apache.org:8020/apps/hbase/staging", - "hbase.regionserver.handler.count": "60", - "hbase.regionserver.global.memstore.lowerLimit": "0.38", - "hbase.hregion.memstore.block.multiplier": "2", - "hbase.hregion.memstore.flush.size": "134217728", - "hbase.superuser": "hbase", - "hbase.zookeeper.property.clientPort": "2181", - "hbase.regionserver.global.memstore.upperLimit": "0.4", - "zookeeper.session.timeout": "30000", - "hbase.tmp.dir": "/hadoop/hbase", - "hbase.hregion.max.filesize": "10737418240", - "hfile.block.cache.size": "0.40", - "hbase.security.authentication": "simple", - "hbase.defaults.for.version.skip": "true", - "hbase.zookeeper.quorum": "c6402.ambari.apache.org", - "zookeeper.znode.parent": "/hbase-unsecure", - "hbase.hstore.blockingStoreFiles": "10", - "hbase.hregion.majorcompaction": "86400000", - "hbase.security.authorization": "false", - "hbase.cluster.distributed": "true", - "hbase.hregion.memstore.mslab.enabled": "true", - "hbase.client.scanner.caching": "100", - "hbase.zookeeper.useMulti": "true", - "hbase.regionserver.info.port": "0", - "hbase.master.info.port": "60010" - }, - "core-site": { - "io.serializations": "org.apache.hadoop.io.serializer.WritableSerialization", - "gluster.daemon.user": "null", - "fs.trash.interval": "360", - "hadoop.security.authentication": "simple", - "io.compression.codecs": "org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec", - "mapreduce.jobtracker.webinterface.trusted": "false", - "fs.AbstractFileSystem.glusterfs.impl": "null", - "fs.defaultFS": "hdfs://c6402.ambari.apache.org:8020", - "ipc.client.connect.max.retries": "50", - "ipc.client.idlethreshold": "8000", - "io.file.buffer.size": "131072", - "hadoop.security.authorization": "false", - "hadoop.security.auth_to_local": "\n RULE:[2:$1@$0]([rn]m@.*)s/.*/yarn/\n RULE:[2:$1@$0](jhs@.*)s/.*/mapred/\n RULE:[2:$1@$0]([nd]n@.*)s/.*/hdfs/\n RULE:[2:$1@$0](hm@.*)s/.*/hbase/\n RULE:[2:$1@$0](rs@.*)s/.*/hbase/\n DEFAULT", - "ipc.client.connection.maxidletime": "30000" - } - }, - "commandId": "2-2" -} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command_template.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command_template.json deleted file mode 100644 index da06c13..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command_template.json +++ /dev/null @@ -1,168 +0,0 @@ -{ - "roleCommand": "{{COMMAND}}", - "clusterName": "{{CLUSTER_NAME}}", - "hostname": "{{HOST_NAME}}", - "hostLevelParams": { - "java_home": "/usr/jdk64/jdk1.7.0_45" - }, - "commandType": "EXECUTION_COMMAND", - "roleParams": {}, - "serviceName": "{{SERVICE_NAME}}", - "role": "{{ROLE_NAME}}", - "commandParams": {}, - "taskId": "{{TASK_ID}}", - "public_hostname": "{{HOST_NAME}}", - "configurations": { - "hbase-log4j": { - "log4j.threshold": "ALL", - "log4j.rootLogger": "${hbase.root.logger}", - "log4j.logger.org.apache.zookeeper": "INFO", - "log4j.logger.org.apache.hadoop.hbase": "DEBUG", - "log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher": "INFO", - "log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil": "INFO", - "log4j.category.SecurityLogger": "${hbase.security.logger}", - "log4j.appender.console": "org.apache.log4j.ConsoleAppender", - "log4j.appender.console.target": "System.err", - "log4j.appender.console.layout": "org.apache.log4j.PatternLayout", - "log4j.appender.console.layout.ConversionPattern": "%d{ISO8601} %-5p [%t] %c{2}: %m%n", - "log4j.appender.RFAS": "org.apache.log4j.RollingFileAppender", - "log4j.appender.RFAS.layout": "org.apache.log4j.PatternLayout", - "log4j.appender.RFAS.layout.ConversionPattern": "%d{ISO8601} %p %c: %m%n", - "log4j.appender.RFAS.MaxFileSize": "${hbase.security.log.maxfilesize}", - "log4j.appender.RFAS.MaxBackupIndex": "${hbase.security.log.maxbackupindex}", - "log4j.appender.RFAS.File": "${hbase.log.dir}/${hbase.security.log.file}", - "log4j.appender.RFA": "org.apache.log4j.RollingFileAppender", - "log4j.appender.RFA.layout": "org.apache.log4j.PatternLayout", - "log4j.appender.RFA.layout.ConversionPattern": "%d{ISO8601} %-5p [%t] %c{2}: %m%n", - "log4j.appender.RFA.MaxFileSize": "${hbase.log.maxfilesize}", - "log4j.appender.RFA.MaxBackupIndex": "${hbase.log.maxbackupindex}", - "log4j.appender.RFA.File": "${hbase.log.dir}/${hbase.log.file}", - "log4j.appender.NullAppender": "org.apache.log4j.varia.NullAppender", - "log4j.appender.DRFA": "org.apache.log4j.DailyRollingFileAppender", - "log4j.appender.DRFA.layout": "org.apache.log4j.PatternLayout", - "log4j.appender.DRFA.layout.ConversionPattern": "%d{ISO8601} %-5p [%t] %c{2}: %m%n", - "log4j.appender.DRFA.File": "${hbase.log.dir}/${hbase.log.file}", - "log4j.appender.DRFA.DatePattern": ".yyyy-MM-dd", - "log4j.additivity.SecurityLogger": "false", - "hbase.security.logger": "INFO,console", - "hbase.security.log.maxfilesize": "256MB", - "hbase.security.log.maxbackupindex": "20", - "hbase.security.log.file": "SecurityAuth.audit", - "hbase.root.logger": "INFO,console", - "hbase.log.maxfilesize": "256MB", - "hbase.log.maxbackupindex": "20", - "hbase.log.file": "hbase.log", - "hbase.log.dir": "." - }, - "global": { - "hbase_root": "{{HBASE_HOME}}", - "hbase_pid_dir": "{{PID_DIR}}", - "proxyuser_group": "users", - "syncLimit": "5", - "hbase_regionserver_heapsize": "{{REGION_SERVER_HEAP_SIZE}}", - "rca_enabled": "false", - "tickTime": "2000", - "hbase_master_heapsize": "{{MASTER_HEAP_SIZE}}", - "initLimit": "10", - "user_group": "{{GROUP_NAME}}", - "hbase_user": "{{USER_NAME}}", - "hbase_log_dir": "{{LOG_DIR}}" - }, - "hdfs-site": { - "dfs.namenode.checkpoint.period": "21600", - "dfs.namenode.avoid.write.stale.datanode": "true", - "dfs.namenode.checkpoint.txns": "1000000", - "dfs.block.access.token.enable": "true", - "dfs.support.append": "true", - "dfs.datanode.address": "0.0.0.0:${ambari.dfs.datanode.port}", - "dfs.cluster.administrators": " hdfs", - "dfs.replication": "3", - "ambari.dfs.datanode.http.port": "50075", - "dfs.datanode.balance.bandwidthPerSec": "6250000", - "dfs.namenode.safemode.threshold-pct": "1.0f", - "dfs.namenode.checkpoint.edits.dir": "${dfs.namenode.checkpoint.dir}", - "dfs.permissions.enabled": "true", - "dfs.client.read.shortcircuit": "true", - "dfs.namenode.https-address": "{{NAMENODE_HTTPS_ADDRESS}}", - "dfs.journalnode.edits.dir": "/grid/0/hdfs/journal", - "dfs.blocksize": "134217728", - "dfs.datanode.max.transfer.threads": "1024", - "dfs.datanode.du.reserved": "1073741824", - "dfs.webhdfs.enabled": "true", - "dfs.namenode.handler.count": "100", - "dfs.namenode.checkpoint.dir": "/hadoop/hdfs/namesecondary", - "fs.permissions.umask-mode": "022", - "dfs.datanode.http.address": "0.0.0.0:${ambari.dfs.datanode.http.port}", - "dfs.datanode.ipc.address": "0.0.0.0:8010", - "dfs.datanode.data.dir": "/hadoop/hdfs/data", - "dfs.namenode.http-address": "{{NAMENODE_HTTP_ADDRESS}}", - "dfs.blockreport.initialDelay": "120", - "dfs.datanode.failed.volumes.tolerated": "0", - "dfs.namenode.accesstime.precision": "0", - "ambari.dfs.datanode.port": "50010", - "dfs.namenode.avoid.read.stale.datanode": "true", - "dfs.namenode.secondary.http-address": "c6402.ambari.apache.org:50090", - "dfs.namenode.stale.datanode.interval": "30000", - "dfs.heartbeat.interval": "3", - "dfs.client.read.shortcircuit.streams.cache.size": "4096", - "dfs.permissions.superusergroup": "hdfs", - "dfs.https.port": "50470", - "dfs.journalnode.http-address": "0.0.0.0:8480", - "dfs.domain.socket.path": "/var/lib/hadoop-hdfs/dn_socket", - "dfs.namenode.write.stale.datanode.ratio": "1.0f", - "dfs.hosts.exclude": "/etc/hadoop/conf/dfs.exclude", - "dfs.datanode.data.dir.perm": "750", - "dfs.namenode.name.dir.restore": "true", - "dfs.replication.max": "50", - "dfs.namenode.name.dir": "/hadoop/hdfs/namenode" - }, - "hbase-site": { - "hbase.hstore.flush.retries.number": "120", - "hbase.client.keyvalue.maxsize": "10485760", - "hbase.hstore.compactionThreshold": "3", - "hbase.rootdir": "{{HBASE_ROOT_DIR}}", - "hbase.stagingdir": "{{HBASE_STAGING_DIR}}", - "hbase.regionserver.handler.count": "60", - "hbase.regionserver.global.memstore.lowerLimit": "0.38", - "hbase.hregion.memstore.block.multiplier": "2", - "hbase.hregion.memstore.flush.size": "134217728", - "hbase.superuser": "{{HBASE_SUPERUSER}}", - "hbase.zookeeper.property.clientPort": "{{ZK_CLIENT_PORT}}", - "hbase.regionserver.global.memstore.upperLimit": "0.4", - "zookeeper.session.timeout": "30000", - "hbase.tmp.dir": "/hadoop/hbase", - "hbase.hregion.max.filesize": "10737418240", - "hfile.block.cache.size": "0.40", - "hbase.security.authentication": "simple", - "hbase.defaults.for.version.skip": "true", - "hbase.zookeeper.quorum": "{{ZK_HOSTS}}", - "zookeeper.znode.parent": "{{ZK_NODE_PARENT}}", - "hbase.hstore.blockingStoreFiles": "10", - "hbase.hregion.majorcompaction": "86400000", - "hbase.security.authorization": "false", - "hbase.cluster.distributed": "true", - "hbase.hregion.memstore.mslab.enabled": "true", - "hbase.client.scanner.caching": "100", - "hbase.zookeeper.useMulti": "true", - "hbase.regionserver.info.port": "{{REGION_SERVER_INFO_PORT}}", - "hbase.master.info.port": "{{MASTER_INFO_PORT}}" - }, - "core-site": { - "io.serializations": "org.apache.hadoop.io.serializer.WritableSerialization", - "gluster.daemon.user": "null", - "fs.trash.interval": "360", - "hadoop.security.authentication": "simple", - "io.compression.codecs": "org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec", - "mapreduce.jobtracker.webinterface.trusted": "false", - "fs.AbstractFileSystem.glusterfs.impl": "null", - "fs.defaultFS": "{{DEFAULT_FS}}", - "ipc.client.connect.max.retries": "50", - "ipc.client.idlethreshold": "8000", - "io.file.buffer.size": "131072", - "hadoop.security.authorization": "false", - "hadoop.security.auth_to_local": "\n RULE:[2:$1@$0]([rn]m@.*)s/.*/yarn/\n RULE:[2:$1@$0](jhs@.*)s/.*/mapred/\n RULE:[2:$1@$0]([nd]n@.*)s/.*/hdfs/\n RULE:[2:$1@$0](hm@.*)s/.*/hbase/\n RULE:[2:$1@$0](rs@.*)s/.*/hbase/\n DEFAULT", - "ipc.client.connection.maxidletime": "30000" - } - }, - "commandId": "{{COMMAND_ID}}" -} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/role-node.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/role-node.xml deleted file mode 100644 index aff1e05..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/role-node.xml +++ /dev/null @@ -1,65 +0,0 @@ - - - - - - - - role.name - node - - - - role.instances - 1 - - - - role.priority - 1 - - - - role.placement.policy - 2 - - - - yarn.memory - 256 - - - - yarn.vcores - 1 - - - - jvm.heapsize - 256M - - - - env.MALLOC_ARENA_MAX - 4 - - - diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/dynamic/application.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/dynamic/application.properties deleted file mode 100644 index d9b42de..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/dynamic/application.properties +++ /dev/null @@ -1,25 +0,0 @@ -# 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. -# - -# gets updated at build time -application.name=${pom.name} -application.version=${pom.version} -application.build=${buildNumber} -application.build.java.version=${java.version} -application.build.user=${user.name} -application.build.info=${pom.name}-${pom.version} Built against commit# ${buildNumber} on Java ${java.version} by ${user.name} -hadoop.build.info=${hadoop.version} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/appconf.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/appconf.json deleted file mode 100644 index 81239a2..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/appconf.json +++ /dev/null @@ -1,19 +0,0 @@ -{ - "schema": "http://example.org/specification/v2.0.0", - - "metadata": { - - - }, - - "global": { - "env.MALLOC_ARENA_MAX": "4" - }, - - "components": { - "slider-appmaster" : { - "jvm.heapsize": "256M" - } - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/internal.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/internal.json deleted file mode 100644 index 2367d8f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/internal.json +++ /dev/null @@ -1,17 +0,0 @@ -{ - "schema": "http://example.org/specification/v2.0.0", - - "metadata": { - }, - - "global": { - "internal.container.failure.shortlife": "60000", - "internal.container.failure.threshold": "5", - "slider.cluster.directory.permissions": "0770", - "slider.data.directory.permissions": "0770" - }, - - "components": { - - } -} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/resources.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/resources.json deleted file mode 100644 index 478ab7e..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/resources.json +++ /dev/null @@ -1,18 +0,0 @@ -{ - "schema": "http://example.org/specification/v2.0.0", - - "metadata": { - - }, - - "global": { - }, - - "components": { - "slider-appmaster": { - "yarn.component.instances": "1", - "yarn.vcores": "1", - "yarn.memory": "1024" - } - } -} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/webapps/slideram/.keep b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/webapps/slideram/.keep deleted file mode 100644 index e69de29..0000000 diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/scripts/slider_keytabs.sh b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/scripts/slider_keytabs.sh deleted file mode 100644 index f0a8fc2..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/scripts/slider_keytabs.sh +++ /dev/null @@ -1,67 +0,0 @@ -#!/usr/bin/env bash - -# 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 script exists to create the keytab set for a node on the cluster -# including hbase and ZK alongside then YARN cores. - -# usage -# keytabs -# validate the args - -num_vars=$# -if [[ $num_vars < 2 ]] -then - echo "Usage: $0 " - exit -2 -fi - -realm="$1" -hostname="$2" -dest="." - -kadmin=kadmin.local - -${kadmin} < empty string - (action, param) = defn - if KEY_DEF == action : - (key, val) = split_to_keyval_tuple(param) - apply_action(conf, KEY_DEF, key, val) - - return conf - - -#def parse_args(conf, args) : -# """ -# split an arg string, parse the jfiles & merge over the conf -# (configuration, args[]) -> (conf', stripped, jfiles[]) -# """ -# (jfiles, stripped) = extract_jfiles(args) -# -# actions = extract_args(args) -# jfiles = actions[KEY_JFILE] -# conf = merge_jfile_list(conf, jfiles) -# return conf, actions - - -def print_conf(conf) : - """ dump the configuration to the console - """ - print "{" - for (key, val) in conf.items() : - if type(val) == dict : - print key - print_conf(val) - else : - print "" + key + " => " + str(val) - print "}" - - -def list_to_str(l, spacer) : - result = "" - for elt in l : - if len(result) > 0 : - result += spacer - result += elt - return result - - -def list_to_hxml_str(l) : - return list_to_str(l, ",") - - -def export_kv_xml(output, key, value) : - line = "" + key + "" + str(value) + "\n" - print line - output.write(line) - - -def export_to_hadoop_xml(output, conf) : - """ export the conf to hadoop XML - dictionaries are skipped. - """ - output.write("\n") - for (key, value) in conf.items() : - if type(value) is list : - # list print - export_kv_xml(output, key, list_to_hxml_str(value)) - else : - if type(value) is dict : - print "skipping dict " + key - else : - export_kv_xml(output, key, value) - output.write("\n") - - -def start(conf, stripped_args) : - """ - start the process by grabbing exec/args for the arguments - """ - ex = conf["exec"] - args = [] - jsonargs = get(ex, "args", []) - args.extend(jsonargs) - args.extend(stripped_args) - classname = get(ex, "classname", "") - if not len(classname) : - raise Exception, "No classname supplied" - classname = get(ex, "classname", "") - commandline = ["java"] - classpath = [] - jvmargs = [] - commandline.extend(jvmargs) - commandline.append("-classpath") - commandline.append(list_to_str(classpath, ":")) - commandline.append("org.apache.hadoop.yarn.service.launcher.ServiceLauncher") - commandline.append(classname) - commandline.extend(args) - print "ready to exec : %s" % commandline - - -def main() : -# (conf, stripped, jfiles) = parse_args({}, sys.argv[1 :]) - actions = extract_args(sys.argv[1 :]) - jfiles = actions[KEY_JFILE] - conf = merge_jfile_list({}, jfiles) - apply_local_definitions(conf, actions[KEY_DEF]) - exec_args = actions[KEY_ARGS] - - print_conf(conf) - # if len(stripped) > 0 : - #got an output file - # filename = stripped[0] - # print "Writing XML configuration to " + filename - # output = open(filename, "w") - # export_to_hadoop_xml(output, conf["site"]) - start(conf, exec_args) - - -if __name__ == "__main__" : - main() - - diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/site/site.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/site/site.xml deleted file mode 100644 index 3b5df7a..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/site/site.xml +++ /dev/null @@ -1,26 +0,0 @@ - - - - - - - - - - \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java index 9746d33..4fa81ee 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java @@ -38,14 +38,13 @@ import org.apache.hadoop.yarn.client.api.async.NMClientAsync; import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.proto.*; import org.apache.hadoop.yarn.proto.ClientAMProtocol; +import org.apache.hadoop.yarn.service.api.records.Application; import org.apache.hadoop.yarn.service.component.Component; import org.apache.hadoop.yarn.service.component.ComponentState; -import org.apache.slider.api.resource.Application; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.core.exceptions.BadClusterStateException; -import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders; +import org.apache.hadoop.yarn.service.exceptions.BadClusterStateException; +import org.apache.hadoop.yarn.service.registry.YarnRegistryViewForProviders; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; import java.io.IOException; import java.util.Collections; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/ServiceTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/ServiceTestUtils.java index ea75a90..73172bf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/ServiceTestUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/ServiceTestUtils.java @@ -18,12 +18,18 @@ package org.apache.hadoop.yarn.service; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.api.resource.Resource; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.api.records.Resource; +import org.apache.hadoop.yarn.service.utils.JsonSerDeser; +import org.codehaus.jackson.map.PropertyNamingStrategy; public class ServiceTestUtils { + public static final JsonSerDeser JSON_SER_DESER = + new JsonSerDeser<>(Application.class, + PropertyNamingStrategy.CAMEL_CASE_TO_LOWER_CASE_WITH_UNDERSCORES); + // Example service definition // 2 components, each of which has 2 containers. protected Application createExampleApplication() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java index d99e30e..1a22875 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java @@ -21,15 +21,14 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.registry.client.api.RegistryConstants; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Artifact; -import org.apache.slider.api.resource.Component; -import org.apache.slider.api.resource.Resource; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.core.persist.JsonSerDeser; -import org.apache.slider.util.RestApiConstants; -import org.apache.slider.util.RestApiErrorMessages; +import org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.api.records.Artifact; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.api.records.Resource; +import org.apache.hadoop.yarn.service.utils.JsonSerDeser; import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -41,15 +40,10 @@ import java.util.Collection; import java.util.List; -import static org.apache.slider.util.RestApiConstants.DEFAULT_COMPONENT_NAME; -import static org.apache.slider.util.RestApiConstants.DEFAULT_UNLIMITED_LIFETIME; -import static org.apache.slider.util.RestApiErrorMessages.*; -import static org.apache.slider.util.RestApiErrorMessages.ERROR_CONTAINERS_COUNT_INVALID; -import static org.apache.slider.util.RestApiErrorMessages.ERROR_RESOURCE_PROFILE_NOT_SUPPORTED_YET; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.createNiceMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; +import static org.apache.hadoop.yarn.service.conf.RestApiConstants.DEFAULT_COMPONENT_NAME; +import static org.apache.hadoop.yarn.service.conf.RestApiConstants.DEFAULT_UNLIMITED_LIFETIME; +import static org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages.*; +import static org.easymock.EasyMock.*; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -136,7 +130,7 @@ public void testResourceValidation() throws Exception { } catch (IllegalArgumentException e) { assertEquals(String.format( RestApiErrorMessages.ERROR_RESOURCE_FOR_COMP_INVALID, - RestApiConstants.DEFAULT_COMPONENT_NAME), e.getMessage()); + DEFAULT_COMPONENT_NAME), e.getMessage()); } // memory not specified @@ -148,7 +142,7 @@ public void testResourceValidation() throws Exception { } catch (IllegalArgumentException e) { assertEquals(String.format( RestApiErrorMessages.ERROR_RESOURCE_MEMORY_FOR_COMP_INVALID, - RestApiConstants.DEFAULT_COMPONENT_NAME), e.getMessage()); + DEFAULT_COMPONENT_NAME), e.getMessage()); } // invalid no of cpus @@ -161,7 +155,7 @@ public void testResourceValidation() throws Exception { } catch (IllegalArgumentException e) { assertEquals(String.format( RestApiErrorMessages.ERROR_RESOURCE_CPUS_FOR_COMP_INVALID_RANGE, - RestApiConstants.DEFAULT_COMPONENT_NAME), e.getMessage()); + DEFAULT_COMPONENT_NAME), e.getMessage()); } // number of containers not specified @@ -183,7 +177,7 @@ public void testResourceValidation() throws Exception { } catch (IllegalArgumentException e) { assertEquals(String.format(RestApiErrorMessages .ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_FOR_COMP_NOT_SUPPORTED, - RestApiConstants.DEFAULT_COMPONENT_NAME), + DEFAULT_COMPONENT_NAME), e.getMessage()); } @@ -476,8 +470,7 @@ public void testDependencySorting() throws IOException { @Test public void testInvalidComponent() throws IOException { SliderFileSystem sfs = initMock(null); - testComponent(sfs, false); - testComponent(sfs, true); + testComponent(sfs); } @Test @@ -496,17 +489,15 @@ public void testValidateCompName() { } } - private static void testComponent(SliderFileSystem sfs, boolean unique) + private static void testComponent(SliderFileSystem sfs) throws IOException { int maxLen = RegistryConstants.MAX_FQDN_LABEL_LENGTH; - if (unique) { - assertEquals(19, Long.toString(Long.MAX_VALUE).length()); - maxLen = maxLen - Long.toString(Long.MAX_VALUE).length(); - } + assertEquals(19, Long.toString(Long.MAX_VALUE).length()); + maxLen = maxLen - Long.toString(Long.MAX_VALUE).length(); + String compName = LEN_64_STR.substring(0, maxLen + 1); Application app = createValidApplication(null); - app.addComponent(createValidComponent(compName).uniqueComponentSupport( - unique)); + app.addComponent(createValidComponent(compName)); // invalid component name fails if dns is enabled try { @@ -526,8 +517,7 @@ private static void testComponent(SliderFileSystem sfs, boolean unique) compName = LEN_64_STR.substring(0, maxLen); app = createValidApplication(null); - app.addComponent(createValidComponent(compName).uniqueComponentSupport( - unique)); + app.addComponent(createValidComponent(compName)); // does not fail try { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java index 28105b2..a36e0b4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java @@ -18,7 +18,6 @@ package org.apache.hadoop.yarn.service; -import com.google.common.base.Supplier; import org.apache.commons.io.FileUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -36,17 +35,16 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.MiniYARNCluster; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.api.records.Container; +import org.apache.hadoop.yarn.service.api.records.ContainerState; import org.apache.hadoop.yarn.service.client.ServiceClient; +import org.apache.hadoop.yarn.service.conf.YarnServiceConf; +import org.apache.hadoop.yarn.service.exceptions.SliderException; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin; import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree; -import org.apache.slider.api.InternalKeys; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.api.resource.Container; -import org.apache.slider.api.resource.ContainerState; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.core.exceptions.SliderException; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -71,8 +69,8 @@ import static org.apache.hadoop.registry.client.api.RegistryConstants.KEY_REGISTRY_ZK_QUORUM; import static org.apache.hadoop.yarn.api.records.YarnApplicationState.FINISHED; import static org.apache.hadoop.yarn.conf.YarnConfiguration.*; -import static org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys.KEY_AM_RESOURCE_MEM; -import static org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys.KEY_SLIDER_BASE_PATH; +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.AM_RESOURCE_MEM; +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.YARN_SERVICE_BASE_PATH; /** * End to end tests to test deploying services with MiniYarnCluster and a in-JVM @@ -122,8 +120,8 @@ private void setupInternal(int numNodeManager) conf.setBoolean(TIMELINE_SERVICE_ENABLED, false); conf.setInt(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 100); conf.setLong(DEBUG_NM_DELETE_DELAY_SEC, 60000); - conf.setLong(KEY_AM_RESOURCE_MEM, 526); - conf.setLong(InternalKeys.MONITOR_INTERVAL, 5); + conf.setLong(AM_RESOURCE_MEM, 526); + conf.setLong(YarnServiceConf.READINESS_CHECK_INTERVAL, 5); // Disable vmem check to disallow NM killing the container conf.setBoolean(NM_VMEM_CHECK_ENABLED, false); conf.setBoolean(NM_PMEM_CHECK_ENABLED, false); @@ -143,7 +141,7 @@ private void setupInternal(int numNodeManager) basedir.mkdirs(); } - conf.set(KEY_SLIDER_BASE_PATH, basedir.getAbsolutePath()); + conf.set(YARN_SERVICE_BASE_PATH, basedir.getAbsolutePath()); if (yarnCluster == null) { yarnCluster = @@ -267,7 +265,7 @@ public void testCreateFlexStopDestroyService() throws Exception { // stop the service LOG.info("Stop the service"); - client.actionStop(exampleApp.getName()); + client.actionStop(exampleApp.getName(), true); ApplicationReport report = client.getYarnClient() .getApplicationReport(ApplicationId.fromString(exampleApp.getId())); // AM unregisters with RM successfully @@ -303,7 +301,7 @@ public void testComponentStartOrder() throws Exception { // check that containers for compa are launched before containers for compb checkContainerLaunchDependencies(client, exampleApp, "compa", "compb"); - client.actionStop(exampleApp.getName()); + client.actionStop(exampleApp.getName(), true); client.actionDestroy(exampleApp.getName()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/client/TestBuildExternalComponents.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/client/TestBuildExternalComponents.java index 4bc9f26..a22c000 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/client/TestBuildExternalComponents.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/client/TestBuildExternalComponents.java @@ -20,11 +20,11 @@ import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.service.api.records.Component; import org.apache.hadoop.yarn.service.conf.ExampleAppJson; -import org.apache.slider.api.resource.Component; import org.apache.hadoop.yarn.service.client.params.ClientArgs; -import org.apache.slider.common.tools.SliderFileSystem; import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -37,7 +37,7 @@ import java.util.Set; import static org.apache.hadoop.yarn.service.client.params.Arguments.ARG_APPDEF; -import static org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys.KEY_SLIDER_BASE_PATH; +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.YARN_SERVICE_BASE_PATH; /** * Test for building / resolving components of type APPLICATION. @@ -87,7 +87,7 @@ public void setup() throws IOException { } else { basedir.mkdirs(); } - conf.set(KEY_SLIDER_BASE_PATH, basedir.getAbsolutePath()); + conf.set(YARN_SERVICE_BASE_PATH, basedir.getAbsolutePath()); } @After diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java index 1f07301..20c06ab 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java @@ -20,14 +20,15 @@ import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.service.ClientAMProtocol; +import org.apache.hadoop.yarn.service.api.records.Component; import org.apache.hadoop.yarn.service.client.params.ClientArgs; import org.apache.hadoop.yarn.service.conf.ExampleAppJson; -import org.apache.slider.api.resource.Component; -import org.apache.slider.common.tools.SliderFileSystem; import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -38,7 +39,7 @@ import java.util.List; import static org.apache.hadoop.yarn.service.client.params.Arguments.ARG_APPDEF; -import static org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys.KEY_SLIDER_BASE_PATH; +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.YARN_SERVICE_BASE_PATH; import static org.mockito.Mockito.mock; public class TestServiceCLI { @@ -59,7 +60,7 @@ private void buildApp(String appName, String appDef) throws Throwable { @Before public void setup() throws Throwable { basedir = new File("target", "apps"); - conf.set(KEY_SLIDER_BASE_PATH, basedir.getAbsolutePath()); + conf.set(YARN_SERVICE_BASE_PATH, basedir.getAbsolutePath()); fs = new SliderFileSystem(conf); if (basedir.exists()) { FileUtils.deleteDirectory(basedir); @@ -71,7 +72,11 @@ public void setup() throws Throwable { cli = new ServiceCLI() { @Override protected void createServiceClient() { client = new ServiceClient() { - @Override protected ClientAMProtocol connectToAM(String appName) + @Override protected ClientAMProtocol getAMProxy(String appName, + ApplicationReport report) throws IOException { + return mock(ClientAMProtocol.class); + } + @Override protected ClientAMProtocol getAMProxy(String appName) throws IOException, YarnException { return mock(ClientAMProtocol.class); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/ExampleAppJson.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/ExampleAppJson.java index 9791976..9e13200 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/ExampleAppJson.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/ExampleAppJson.java @@ -18,13 +18,14 @@ package org.apache.hadoop.yarn.service.conf; -import org.apache.slider.api.resource.Application; + +import org.apache.hadoop.yarn.service.api.records.Application; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import static org.apache.slider.utils.SliderTestUtils.JSON_SER_DESER; +import static org.apache.hadoop.yarn.service.ServiceTestUtils.JSON_SER_DESER; /** * Names of the example configs. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestAppJsonResolve.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestAppJsonResolve.java index 66939a1..954d117 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestAppJsonResolve.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestAppJsonResolve.java @@ -21,14 +21,13 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.ConfigFile; -import org.apache.slider.api.resource.ConfigFile.TypeEnum; -import org.apache.slider.api.resource.Configuration; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.persist.JsonSerDeser; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.api.records.ConfigFile; +import org.apache.hadoop.yarn.service.api.records.Configuration; +import org.apache.hadoop.yarn.service.utils.JsonSerDeser; 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.junit.Assert; import org.junit.Test; import org.slf4j.Logger; @@ -41,18 +40,9 @@ import java.util.Map; import java.util.Set; -import static org.apache.slider.api.InternalKeys.CHAOS_MONKEY_INTERVAL; -import static org.apache.slider.api.InternalKeys.DEFAULT_CHAOS_MONKEY_INTERVAL_DAYS; -import static org.apache.slider.api.InternalKeys.DEFAULT_CHAOS_MONKEY_INTERVAL_HOURS; -import static org.apache.slider.api.InternalKeys.DEFAULT_CHAOS_MONKEY_INTERVAL_MINUTES; -import static org.apache.hadoop.yarn.service.conf.ExampleAppJson.APP_JSON; -import static org.apache.hadoop.yarn.service.conf.ExampleAppJson.EXTERNAL_JSON_1; -import static org.apache.hadoop.yarn.service.conf.ExampleAppJson.OVERRIDE_JSON; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.createNiceMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.reset; +import static org.apache.hadoop.yarn.service.conf.ExampleAppJson.*; +import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.*; +import static org.easymock.EasyMock.*; /** * Test global configuration resolution. @@ -115,9 +105,9 @@ public void testOverride() throws Throwable { Map props = new HashMap<>(); props.put("k1", "overridden"); props.put("k2", "v2"); - files.add(new ConfigFile().destFile("file1").type(TypeEnum + files.add(new ConfigFile().destFile("file1").type(ConfigFile.TypeEnum .PROPERTIES).props(props)); - files.add(new ConfigFile().destFile("file2").type(TypeEnum + files.add(new ConfigFile().destFile("file2").type(ConfigFile.TypeEnum .XML).props(Collections.singletonMap("k3", "v3"))); assertTrue(files.contains(simple.getFiles().get(0))); assertTrue(files.contains(simple.getFiles().get(1))); @@ -132,9 +122,9 @@ public void testOverride() throws Throwable { props.put("k1", "v1"); files.clear(); - files.add(new ConfigFile().destFile("file1").type(TypeEnum + files.add(new ConfigFile().destFile("file1").type(ConfigFile.TypeEnum .PROPERTIES).props(props)); - files.add(new ConfigFile().destFile("file2").type(TypeEnum + files.add(new ConfigFile().destFile("file2").type(ConfigFile.TypeEnum .XML).props(Collections.singletonMap("k3", "v3"))); assertTrue(files.contains(master.getFiles().get(0))); @@ -208,7 +198,7 @@ public void testOverrideExternalConfiguration() throws IOException { assertEquals("a", simple.getProperty("g1")); assertEquals("b", simple.getProperty("g2")); assertEquals("60", - simple.getProperty("internal.chaos.monkey.interval.seconds")); + simple.getProperty("yarn.service.failure-count-reset.window")); master = orig.getComponent("master").getConfiguration(); assertEquals(5, master.getProperties().size()); @@ -217,7 +207,7 @@ public void testOverrideExternalConfiguration() throws IOException { assertEquals("b", master.getProperty("g2")); assertEquals("is-overridden", master.getProperty("g3")); assertEquals("60", - simple.getProperty("internal.chaos.monkey.interval.seconds")); + simple.getProperty("yarn.service.failure-count-reset.window")); Configuration worker = orig.getComponent("worker").getConfiguration(); LOG.info("worker = {}", worker); @@ -226,27 +216,9 @@ public void testOverrideExternalConfiguration() throws IOException { assertEquals("overridden-by-worker", worker.getProperty("g1")); assertEquals("b", worker.getProperty("g2")); assertEquals("60", - worker.getProperty("internal.chaos.monkey.interval.seconds")); + worker.getProperty("yarn.service.failure-count-reset.window")); other = orig.getComponent("other").getConfiguration(); assertEquals(0, other.getProperties().size()); } - - @Test - public void testTimeIntervalLoading() throws Throwable { - Application orig = ExampleAppJson.loadResource(APP_JSON); - - Configuration conf = orig.getConfiguration(); - long s = conf.getPropertyLong( - CHAOS_MONKEY_INTERVAL + SliderUtils.SECONDS, - 0); - assertEquals(60, s); - long monkeyInterval = SliderUtils.getTimeRange(conf, - CHAOS_MONKEY_INTERVAL, - DEFAULT_CHAOS_MONKEY_INTERVAL_DAYS, - DEFAULT_CHAOS_MONKEY_INTERVAL_HOURS, - DEFAULT_CHAOS_MONKEY_INTERVAL_MINUTES, - 0); - assertEquals(60L, monkeyInterval); - } -} +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestLoadExampleAppJson.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestLoadExampleAppJson.java index b304901..8310530 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestLoadExampleAppJson.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestLoadExampleAppJson.java @@ -21,9 +21,9 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.slider.api.resource.Application; -import org.apache.slider.common.tools.SliderFileSystem; +import org.apache.hadoop.yarn.service.api.records.Application; import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; +import org.apache.hadoop.yarn.service.utils.SliderFileSystem; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -32,11 +32,8 @@ import java.util.Arrays; import java.util.Collection; -import static org.apache.slider.utils.SliderTestUtils.JSON_SER_DESER; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.createNiceMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; +import static org.apache.hadoop.yarn.service.ServiceTestUtils.JSON_SER_DESER; +import static org.easymock.EasyMock.*; /** * Test loading example resources. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestValidateServiceNames.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestValidateServiceNames.java index 9a8dbee..98c78d3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestValidateServiceNames.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestValidateServiceNames.java @@ -18,7 +18,7 @@ package org.apache.hadoop.yarn.service.conf; -import org.apache.slider.common.tools.SliderUtils; +import org.apache.hadoop.yarn.service.utils.SliderUtils; import org.junit.Assert; import org.junit.Test; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java index 568a066..5b24a1d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java @@ -19,10 +19,9 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.service.api.records.Artifact; +import org.apache.hadoop.yarn.service.api.records.ConfigFile; import org.apache.hadoop.yarn.service.provider.AbstractClientProvider; -import org.apache.slider.api.resource.Artifact; -import org.apache.slider.api.resource.ConfigFile; -import org.apache.slider.api.resource.ConfigFile.TypeEnum; import org.junit.Assert; import org.junit.Test; @@ -70,7 +69,7 @@ public void testConfigFiles() throws IOException { } catch (IllegalArgumentException e) { } - configFile.setType(TypeEnum.TEMPLATE); + configFile.setType(ConfigFile.TypeEnum.TEMPLATE); try { clientProvider.validateConfigFiles(configFiles, mockFs); Assert.fail(EXCEPTION_PREFIX + "empty src_file for type template"); @@ -92,7 +91,7 @@ public void testConfigFiles() throws IOException { } configFile = new ConfigFile(); - configFile.setType(TypeEnum.JSON); + configFile.setType(ConfigFile.TypeEnum.JSON); configFile.setSrcFile(null); configFile.setDestFile("path/destfile2"); configFiles.add(configFile); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestProviderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestProviderFactory.java index 5cb5793..489578d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestProviderFactory.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestProviderFactory.java @@ -18,6 +18,8 @@ package org.apache.hadoop.yarn.service.providers; +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.provider.ProviderFactory; import org.apache.hadoop.yarn.service.provider.defaultImpl.DefaultClientProvider; import org.apache.hadoop.yarn.service.provider.defaultImpl.DefaultProviderFactory; @@ -28,8 +30,7 @@ import org.apache.hadoop.yarn.service.provider.tarball.TarballClientProvider; import org.apache.hadoop.yarn.service.provider.tarball.TarballProviderFactory; import org.apache.hadoop.yarn.service.provider.tarball.TarballProviderService; -import org.apache.slider.api.resource.Artifact; -import org.apache.slider.api.resource.Artifact.TypeEnum; + import org.junit.Test; import static org.junit.Assert.assertTrue; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/servicemonitor/TestServiceMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/servicemonitor/TestServiceMonitor.java index db83cb6..6f5653f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/servicemonitor/TestServiceMonitor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/servicemonitor/TestServiceMonitor.java @@ -20,33 +20,14 @@ package org.apache.hadoop.yarn.service.servicemonitor; import org.apache.commons.io.FileUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; -import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; -import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; -import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.service.MockServiceAM; import org.apache.hadoop.yarn.service.ServiceTestUtils; -import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; -import org.apache.slider.api.InternalKeys; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.core.exceptions.BadClusterStateException; + +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.conf.YarnServiceConf; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -54,15 +35,7 @@ import java.io.File; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; -import java.util.List; - -import static org.mockito.Matchers.anyFloat; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.*; public class TestServiceMonitor extends ServiceTestUtils { @@ -77,7 +50,7 @@ public void setup() throws Exception { } else { basedir.mkdirs(); } - conf.setLong(InternalKeys.MONITOR_INTERVAL, 2); + conf.setLong(YarnServiceConf.READINESS_CHECK_INTERVAL, 2); } @After diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/timelineservice/TestServiceTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/timelineservice/TestServiceTimelinePublisher.java index 476727a..a891df8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/timelineservice/TestServiceTimelinePublisher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/hadoop/yarn/service/timelineservice/TestServiceTimelinePublisher.java @@ -19,8 +19,8 @@ package org.apache.hadoop.yarn.service.timelineservice; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity.Identifier; import org.apache.hadoop.yarn.client.api.TimelineV2Client; @@ -28,19 +28,16 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.service.ServiceContext; -import org.apache.hadoop.yarn.service.ServiceScheduler; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.ApplicationState; -import org.apache.slider.api.resource.Artifact; -import org.apache.slider.api.resource.Component; -import org.apache.slider.api.resource.Container; -import org.apache.slider.api.resource.ContainerState; -import org.apache.slider.api.resource.PlacementPolicy; -import org.apache.slider.api.resource.Resource; -import org.apache.slider.server.appmaster.actions.ActionStopSlider; +import org.apache.hadoop.yarn.service.api.records.Application; +import org.apache.hadoop.yarn.service.api.records.ApplicationState; +import org.apache.hadoop.yarn.service.api.records.Artifact; +import org.apache.hadoop.yarn.service.api.records.Component; +import org.apache.hadoop.yarn.service.api.records.Container; +import org.apache.hadoop.yarn.service.api.records.ContainerState; +import org.apache.hadoop.yarn.service.api.records.PlacementPolicy; +import org.apache.hadoop.yarn.service.api.records.Resource; import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; import org.apache.hadoop.yarn.service.compinstance.ComponentInstanceId; -import org.apache.slider.server.appmaster.state.AppState; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -103,16 +100,9 @@ public void tearDown() throws Exception { @Test public void testServiceAttemptEntity() { - AppState appState = createMockAppState(); - int exitCode = 0; - String message = "Stopped by user"; - ActionStopSlider stopAction = mock(ActionStopSlider.class); - when(stopAction.getExitCode()).thenReturn(exitCode); - when(stopAction.getFinalApplicationStatus()) - .thenReturn(FinalApplicationStatus.SUCCEEDED); - when(stopAction.getMessage()).thenReturn(message); - - serviceTimelinePublisher.serviceAttemptRegistered(appState.getClusterStatus()); + Application application = createMockApplication(); + serviceTimelinePublisher + .serviceAttemptRegistered(application, new YarnConfiguration()); Collection lastPublishedEntities = ((DummyTimelineClient) timelineClient).getLastPublishedEntities(); @@ -123,17 +113,21 @@ public void testServiceAttemptEntity() { .toString()) { verifyComponentTimelineEntity(timelineEntity); } else { - verifyServiceAttemptTimelineEntity(timelineEntity, 0, null, true); + verifyServiceAttemptTimelineEntity(timelineEntity, null, true); } } - serviceTimelinePublisher.serviceAttemptUnregistered(appState, stopAction); + ServiceContext context = new ServiceContext(); + context.attemptId = ApplicationAttemptId + .newInstance(ApplicationId.fromString(application.getId()), 1); + String exitDiags = "service killed"; + serviceTimelinePublisher.serviceAttemptUnregistered(context, exitDiags); lastPublishedEntities = ((DummyTimelineClient) timelineClient).getLastPublishedEntities(); for (TimelineEntity timelineEntity : lastPublishedEntities) { if (timelineEntity.getType() == ServiceTimelineEntityType.SERVICE_ATTEMPT .toString()) { - verifyServiceAttemptTimelineEntity(timelineEntity, exitCode, message, + verifyServiceAttemptTimelineEntity(timelineEntity, exitDiags, false); } } @@ -180,7 +174,7 @@ public void testComponentInstanceEntity() { } private void verifyServiceAttemptTimelineEntity(TimelineEntity timelineEntity, - int exitCode, String message, boolean isRegistedEntity) { + String message, boolean isRegistedEntity) { assertEquals(SERVICEID, timelineEntity.getId()); assertEquals(SERVICE_NAME, timelineEntity.getInfo().get(ServiceTimelineMetricsConstants.NAME)); @@ -190,13 +184,10 @@ private void verifyServiceAttemptTimelineEntity(TimelineEntity timelineEntity, assertEquals(ServiceTimelineEvent.SERVICE_ATTEMPT_REGISTERED.toString(), timelineEntity.getEvents().iterator().next().getId()); } else { - assertEquals("SUCCEEDED", - timelineEntity.getInfo().get(ServiceTimelineMetricsConstants.STATE)); - assertEquals(exitCode, timelineEntity.getInfo() - .get(ServiceTimelineMetricsConstants.EXIT_STATUS_CODE)); + assertEquals("ENDED", + timelineEntity.getInfo().get(ServiceTimelineMetricsConstants.STATE).toString()); assertEquals(message, timelineEntity.getInfo() - .get(ServiceTimelineMetricsConstants.EXIT_REASON)); - + .get(ServiceTimelineMetricsConstants.DIAGNOSTICS_INFO)); assertEquals(2, timelineEntity.getEvents().size()); assertEquals(ServiceTimelineEvent.SERVICE_ATTEMPT_UNREGISTERED.toString(), timelineEntity.getEvents().iterator().next().getId()); @@ -218,23 +209,20 @@ private void verifyComponentTimelineEntity(TimelineEntity entity) { assertEquals("sleep 1", info.get(ServiceTimelineMetricsConstants.LAUNCH_COMMAND)); assertEquals("false", - info.get(ServiceTimelineMetricsConstants.UNIQUE_COMPONENT_SUPPORT)); - assertEquals("false", info.get(ServiceTimelineMetricsConstants.RUN_PRIVILEGED_CONTAINER)); assertEquals("label", info.get(ServiceTimelineMetricsConstants.PLACEMENT_POLICY)); } - private static AppState createMockAppState() { - AppState appState = mock(AppState.class); + private static Application createMockApplication() { Application application = mock(Application.class); when(application.getId()).thenReturn(SERVICEID); when(application.getLaunchTime()).thenReturn(new Date()); when(application.getState()).thenReturn(ApplicationState.STARTED); when(application.getName()).thenReturn(SERVICE_NAME); - when(application.getConfiguration()) - .thenReturn(new org.apache.slider.api.resource.Configuration()); + when(application.getConfiguration()).thenReturn( + new org.apache.hadoop.yarn.service.api.records.Configuration()); Component component = mock(Component.class); Artifact artifact = new Artifact(); @@ -250,19 +238,13 @@ private static AppState createMockAppState() { PlacementPolicy placementPolicy = new PlacementPolicy(); placementPolicy.setLabel("label"); when(component.getPlacementPolicy()).thenReturn(placementPolicy); - when(component.getConfiguration()) - .thenReturn(new org.apache.slider.api.resource.Configuration()); + when(component.getConfiguration()).thenReturn( + new org.apache.hadoop.yarn.service.api.records.Configuration()); List components = new ArrayList(); components.add(component); when(application.getComponents()).thenReturn(components); - when(appState.getClusterStatus()).thenReturn(application); - return appState; - } - - public static void main(String[] args) { - Application application = createMockAppState().getClusterStatus(); - System.out.println(application.getConfiguration()); + return application; } protected static class DummyTimelineClient extends TimelineV2ClientImpl { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/api/TestRPCBinding.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/api/TestRPCBinding.java deleted file mode 100644 index 0c2a2aa..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/api/TestRPCBinding.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.slider.api; - -import org.apache.hadoop.conf.Configuration; -import org.apache.slider.server.appmaster.rpc.RpcBinder; -import org.apache.slider.server.appmaster.rpc.SliderClusterProtocolPB; -import org.junit.Test; - -import java.net.InetSocketAddress; - -import static org.junit.Assert.assertTrue; - -/** - * Tests RPC work. - */ -public class TestRPCBinding { - - //@Test - public void testRegistration() throws Throwable { - Configuration conf = new Configuration(); - RpcBinder.registerSliderAPI(conf); - assertTrue(RpcBinder.verifyBondedToProtobuf(conf, - SliderClusterProtocolPB.class)); - } - - //@Test - public void testGetProxy() throws Throwable { - Configuration conf = new Configuration(); - InetSocketAddress saddr = new InetSocketAddress("127.0.0.1", 9000); - SliderClusterProtocol proxy = - RpcBinder.connectToServer(saddr, null, conf, 1000); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestClientBadArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestClientBadArgs.java deleted file mode 100644 index 7b0586e..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestClientBadArgs.java +++ /dev/null @@ -1,229 +0,0 @@ -/* - * 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.slider.client; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.service.client.params.Arguments; -import org.apache.hadoop.yarn.service.client.params.SliderActions; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.ErrorStrings; -import org.apache.slider.core.exceptions.UsageException; -import org.apache.slider.utils.SliderTestBase; -import org.junit.Ignore; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Arrays; - -/** - * Test the argument parsing/validation logic. - */ -public class TestClientBadArgs extends SliderTestBase { - private static final Logger LOG = - LoggerFactory.getLogger(TestClientBadArgs.class); - - //@Test - public void testNoAction() throws Throwable { - launchExpectingException(SliderClient.class, - createTestConfig(), - "Usage: slider COMMAND", - EMPTY_LIST); - - } - - //@Test - public void testUnknownAction() throws Throwable { - launchExpectingException(SliderClient.class, - createTestConfig(), - "not-a-known-action", - Arrays.asList("not-a-known-action")); - } - - //@Test - public void testActionWithoutOptions() throws Throwable { - launchExpectingException(SliderClient.class, - createTestConfig(), - "Usage: slider build ", - Arrays.asList(SliderActions.ACTION_BUILD)); - } - - //@Test - public void testActionWithoutEnoughArgs() throws Throwable { - launchExpectingException(SliderClient.class, - createTestConfig(), - ErrorStrings.ERROR_NOT_ENOUGH_ARGUMENTS, - Arrays.asList(SliderActions.ACTION_START)); - } - - //@Test - public void testActionWithTooManyArgs() throws Throwable { - launchExpectingException(SliderClient.class, - createTestConfig(), - ErrorStrings.ERROR_TOO_MANY_ARGUMENTS, - Arrays.asList(SliderActions.ACTION_HELP, - "hello, world")); - } - - //@Test - public void testBadImageArg() throws Throwable { - launchExpectingException(SliderClient.class, - createTestConfig(), - "Unknown option: --image", - Arrays.asList(SliderActions.ACTION_HELP, - Arguments.ARG_IMAGE)); - } - - //@Test - public void testRegistryUsage() throws Throwable { - Throwable exception = launchExpectingException(SliderClient.class, - createTestConfig(), - "org.apache.slider.core.exceptions.UsageException: Argument --name " + - "missing", - Arrays.asList(SliderActions.ACTION_REGISTRY)); - assertTrue(exception instanceof UsageException); - LOG.info(exception.toString()); - } - - //@Test - public void testRegistryExportBadUsage1() throws Throwable { - Throwable exception = launchExpectingException(SliderClient.class, - createTestConfig(), - "Expected a value after parameter --getexp", - Arrays.asList(SliderActions.ACTION_REGISTRY, - Arguments.ARG_NAME, - "cl1", - Arguments.ARG_GETEXP)); - assertTrue(exception instanceof BadCommandArgumentsException); - LOG.info(exception.toString()); - } - - //@Test - public void testRegistryExportBadUsage2() throws Throwable { - Throwable exception = launchExpectingException(SliderClient.class, - createTestConfig(), - "Expected a value after parameter --getexp", - Arrays.asList(SliderActions.ACTION_REGISTRY, - Arguments.ARG_NAME, - "cl1", - Arguments.ARG_LISTEXP, - Arguments.ARG_GETEXP)); - assertTrue(exception instanceof BadCommandArgumentsException); - LOG.info(exception.toString()); - } - - //@Test - public void testRegistryExportBadUsage3() throws Throwable { - Throwable exception = launchExpectingException(SliderClient.class, - createTestConfig(), - "Usage: registry", - Arrays.asList(SliderActions.ACTION_REGISTRY, - Arguments.ARG_NAME, - "cl1", - Arguments.ARG_LISTEXP, - Arguments.ARG_GETEXP, - "export1")); - assertTrue(exception instanceof UsageException); - LOG.info(exception.toString()); - } - - //@Test - public void testUpgradeUsage() throws Throwable { - Throwable exception = launchExpectingException(SliderClient.class, - createTestConfig(), - "org.apache.slider.core.exceptions.BadCommandArgumentsException: Not " + - "enough arguments for action: upgrade Expected minimum 1 but got 0", - Arrays.asList(SliderActions.ACTION_UPGRADE)); - assertTrue(exception instanceof BadCommandArgumentsException); - LOG.info(exception.toString()); - } - - public Configuration createTestConfig() { - Configuration configuration = new Configuration(); - configuration.set(YarnConfiguration.RM_ADDRESS, "127.0.0.1:8032"); - return configuration; - } - - @Ignore - //@Test - public void testUpgradeWithTemplateResourcesAndContainersOption() throws - Throwable { - //TODO test upgrade args - String appName = "test_hbase"; - Throwable exception = launchExpectingException(SliderClient.class, - createTestConfig(), - "BadCommandArgumentsException: Option --containers cannot be " - + "specified with --appdef", - Arrays.asList(SliderActions.ACTION_UPGRADE, - appName, - Arguments.ARG_APPDEF, - "/tmp/app.json", - Arguments.ARG_CONTAINERS, - "container_1" - )); - assertTrue(exception instanceof BadCommandArgumentsException); - LOG.info(exception.toString()); - } - - @Ignore - //@Test - public void testUpgradeWithTemplateResourcesAndComponentsOption() throws - Throwable { - //TODO test upgrade args - String appName = "test_hbase"; - Throwable exception = launchExpectingException(SliderClient.class, - createTestConfig(), - "BadCommandArgumentsException: Option --components cannot be " - + "specified with --appdef", - Arrays.asList(SliderActions.ACTION_UPGRADE, - appName, - Arguments.ARG_APPDEF, - "/tmp/app.json", - Arguments.ARG_COMPONENTS, - "HBASE_MASTER" - )); - assertTrue(exception instanceof BadCommandArgumentsException); - LOG.info(exception.toString()); - } - - //@Test - public void testNodesMissingFile() throws Throwable { - Throwable exception = launchExpectingException(SliderClient.class, - createTestConfig(), - "after parameter --out", - Arrays.asList(SliderActions.ACTION_NODES, Arguments.ARG_OUTPUT)); - assertTrue(exception instanceof BadCommandArgumentsException); - } - - //@Test - public void testFlexWithNoComponents() throws Throwable { - Throwable exception = launchExpectingException(SliderClient.class, - new Configuration(), - "Usage: slider flex ", - Arrays.asList( - SliderActions.ACTION_FLEX, - "flex1", - Arguments.ARG_DEFINE, - YarnConfiguration.RM_ADDRESS + "=127.0.0.1:8032" - )); - assertTrue(exception instanceof UsageException); - LOG.info(exception.toString()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestClientBasicArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestClientBasicArgs.java deleted file mode 100644 index b29c581..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestClientBasicArgs.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.slider.client; - -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.service.client.params.Arguments; -import org.apache.hadoop.yarn.service.client.params.ClientArgs; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.main.ServiceLauncher; -import org.apache.slider.utils.SliderTestBase; -import org.junit.Test; - -import java.net.UnknownHostException; -import java.util.Arrays; - -/** - * Test bad argument handling. - */ -public class TestClientBasicArgs extends SliderTestBase { - - /** - * Help should print out help string and then succeed. - * @throws Throwable - */ - //@Test - public void testHelp() throws Throwable { - ServiceLauncher launcher = launch(SliderClient.class, - SliderUtils.createConfiguration(), - Arrays.asList(ClientArgs.ACTION_HELP)); - assertEquals(0, launcher.getServiceExitCode()); - } - - //@Test - public void testNoArgs() throws Throwable { - launchExpectingException(SliderClient.class, - SliderUtils.createConfiguration(), - "Usage: slider COMMAND", - EMPTY_LIST); - } - - //@Test - public void testListUnknownRM() throws Throwable { - try { - YarnConfiguration conf = SliderUtils.createConfiguration(); - conf.setLong(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS, - 1000); - conf.setLong(YarnConfiguration - .RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS, 1000); - ServiceLauncher launcher = launch(SliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_LIST, - "cluster", - Arguments.ARG_MANAGER, - "badhost:8888")); - fail("expected an exception, got a launcher with exit code " + - launcher.getServiceExitCode()); - } catch (UnknownHostException expected) { - //expected - } - - } - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestCommonArgParsing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestCommonArgParsing.java deleted file mode 100644 index 72960fa..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestCommonArgParsing.java +++ /dev/null @@ -1,522 +0,0 @@ -/* - * 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.slider.client; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.api.RoleKeys; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; -import org.apache.slider.common.params.AbstractClusterBuildingActionArgs; -import org.apache.hadoop.yarn.service.client.params.ActionBuildArgs; -import org.apache.hadoop.yarn.service.client.params.ActionCreateArgs; -import org.apache.hadoop.yarn.service.client.params.ActionDestroyArgs; -import org.apache.slider.common.params.ActionExistsArgs; -import org.apache.hadoop.yarn.service.client.params.ActionFlexArgs; -import org.apache.slider.common.params.ActionFreezeArgs; -import org.apache.slider.common.params.ActionListArgs; -import org.apache.slider.common.params.ActionStatusArgs; -import org.apache.slider.common.params.ActionThawArgs; -import org.apache.slider.common.params.ActionUpdateArgs; -import org.apache.hadoop.yarn.service.client.params.ArgOps; -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.SliderActions; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.ErrorStrings; -import org.apache.slider.core.exceptions.SliderException; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -/** - * Test handling of common arguments, specifically how things get split up. - */ -public class TestCommonArgParsing implements SliderActions, Arguments { - - - public static final String CLUSTERNAME = "clustername"; - - //@Test - public void testCreateActionArgs() throws Throwable { - ClientArgs clientArgs = createClientArgs(Arrays.asList(ACTION_CREATE, - "cluster1")); - assertEquals("cluster1", clientArgs.getClusterName()); - } - - //@Test - public void testCreateFailsNoClustername() throws Throwable { - assertParseFails(Arrays.asList(ACTION_CREATE)); - } - - //@Test - public void testCreateFailsTwoClusternames() throws Throwable { - assertParseFails(Arrays.asList( - ACTION_CREATE, - "c1", - "c2" - )); - } - - //@Test - public void testHelp() throws Throwable { - ClientArgs clientArgs = createClientArgs(Arrays.asList(ACTION_HELP)); - assertNull(clientArgs.getClusterName()); - } - - //@Test - public void testSliderBasePath() throws Throwable { - ClientArgs clientArgs = createClientArgs(Arrays.asList(ACTION_LIST, - ARG_BASE_PATH, "/projects/slider/clusters")); - assertEquals(new Path("/projects/slider/clusters"), - clientArgs.getBasePath()); - } - - //@Test - public void testNoSliderBasePath() throws Throwable { - ClientArgs clientArgs = createClientArgs(Arrays.asList(ACTION_LIST)); - assertNull(clientArgs.getBasePath()); - } - - //@Test - public void testListNoClusternames() throws Throwable { - ClientArgs clientArgs = createClientArgs(Arrays.asList(ACTION_LIST)); - assertNull(clientArgs.getClusterName()); - } - - //@Test - public void testListNoClusternamesDefinition() throws Throwable { - ClientArgs clientArgs = createClientArgs(Arrays.asList( - ACTION_LIST, - ARG_DEFINE, - "fs.default.FS=file://localhost" - )); - assertNull(clientArgs.getClusterName()); - } - - //@Test - public void testList1Clustername() throws Throwable { - ClientArgs ca = createClientArgs(Arrays.asList(ACTION_LIST, "cluster1")); - assertEquals("cluster1", ca.getClusterName()); - assertTrue(ca.getCoreAction() instanceof ActionListArgs); - } - - //@Test - public void testListFailsTwoClusternames() throws Throwable { - assertParseFails(Arrays.asList( - ACTION_LIST, - "c1", - "c2" - )); - } - - //@Test - public void testDefinitions() throws Throwable { - ClientArgs ca = createClientArgs(Arrays.asList( - ACTION_CREATE, - CLUSTERNAME, - "-D", "yarn.resourcemanager.principal=yarn/server@LOCAL", - "-D", "dfs.datanode.kerberos.principal=hdfs/server@LOCAL" - )); - Configuration conf = new Configuration(false); - ca.applyDefinitions(conf); - assertEquals(CLUSTERNAME, ca.getClusterName()); - assertNull(conf.get(SliderXmlConfKeys.KEY_SLIDER_BASE_PATH)); - SliderUtils.verifyPrincipalSet(conf, YarnConfiguration.RM_PRINCIPAL); - SliderUtils.verifyPrincipalSet( - conf, - SliderXmlConfKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY); - - } - - //@Test - public void testDefinitionsSettingBaseSliderDir() throws Throwable { - ClientArgs ca = createClientArgs(Arrays.asList( - ACTION_CREATE, - CLUSTERNAME, - "--basepath", "/projects/slider/clusters", - "-D", "yarn.resourcemanager.principal=yarn/server@LOCAL", - "-D", "dfs.datanode.kerberos.principal=hdfs/server@LOCAL" - )); - Configuration conf = new Configuration(false); - ca.applyDefinitions(conf); - assertEquals(CLUSTERNAME, ca.getClusterName()); - assertEquals("/projects/slider/clusters", conf.get(SliderXmlConfKeys - .KEY_SLIDER_BASE_PATH)); - SliderUtils.verifyPrincipalSet(conf, YarnConfiguration.RM_PRINCIPAL); - SliderUtils.verifyPrincipalSet(conf, SliderXmlConfKeys - .DFS_DATANODE_KERBEROS_PRINCIPAL_KEY); - - } - - /** - * Test a start command. - * @throws Throwable - */ - //@Test - public void testComplexThaw() throws Throwable { - ClientArgs ca = createClientArgs(Arrays.asList( - ACTION_START, - "--manager", "rhel:8032", - "--filesystem", "hdfs://rhel:9090", - "-S", "java.security.krb5.realm=LOCAL", - "-S", "java.security.krb5.kdc=rhel", - "-D", "yarn.resourcemanager.principal=yarn/rhel@LOCAL", - "-D", "namenode.resourcemanager.principal=hdfs/rhel@LOCAL", - "cl1" - )); - assertEquals("cl1", ca.getClusterName()); - assertTrue(ca.getCoreAction() instanceof ActionThawArgs); - } - - /** - * Test a force kill command where the app comes at the end of the line. - * @throws Throwable - * - */ - //@Test - public void testStatusSplit() throws Throwable { - - String appId = "application_1381252124398_0013"; - ClientArgs ca = createClientArgs(Arrays.asList( - ACTION_STATUS, - "--manager", "rhel:8032", - "--filesystem", "hdfs://rhel:9090", - "-S", "java.security.krb5.realm=LOCAL", - "-S", "java.security.krb5.kdc=rhel", - "-D", "yarn.resourcemanager.principal=yarn/rhel@LOCAL", - "-D", "namenode.resourcemanager.principal=hdfs/rhel@LOCAL", - appId - )); - assertEquals(appId, ca.getClusterName()); - } - - //@Test - public void testFreezeFailsNoArg() throws Throwable { - assertParseFails(Arrays.asList( - ACTION_STOP - )); - } - - //@Test - public void testFreezeWorks1Arg() throws Throwable { - ClientArgs ca = createClientArgs(Arrays.asList( - ACTION_STOP, - CLUSTERNAME - )); - assertEquals(CLUSTERNAME, ca.getClusterName()); - assertTrue(ca.getCoreAction() instanceof ActionFreezeArgs); - } - - //@Test - public void testFreezeFails2Arg() throws Throwable { - assertParseFails(Arrays.asList( - ACTION_STOP, "cluster", "cluster2" - )); - } - - //@Test - public void testFreezeForceWaitAndMessage() throws Throwable { - ClientArgs ca = createClientArgs(Arrays.asList( - ACTION_STOP, CLUSTERNAME, - ARG_FORCE, - ARG_WAIT, "0", - ARG_MESSAGE, "explanation" - )); - assertEquals(CLUSTERNAME, ca.getClusterName()); - assertTrue(ca.getCoreAction() instanceof ActionFreezeArgs); - ActionFreezeArgs freezeArgs = (ActionFreezeArgs) ca.getCoreAction(); - assertEquals("explanation", freezeArgs.message); - assertTrue(freezeArgs.force); - } - - //@Test - public void testGetStatusWorks1Arg() throws Throwable { - ClientArgs ca = createClientArgs(Arrays.asList( - ACTION_STATUS, - CLUSTERNAME - )); - assertEquals(CLUSTERNAME, ca.getClusterName()); - assertTrue(ca.getCoreAction() instanceof ActionStatusArgs); - } - - //@Test - public void testExistsWorks1Arg() throws Throwable { - ClientArgs ca = createClientArgs(Arrays.asList( - ACTION_EXISTS, - CLUSTERNAME, - ARG_LIVE - )); - assertEquals(CLUSTERNAME, ca.getClusterName()); - assertTrue(ca.getCoreAction() instanceof ActionExistsArgs); - assertTrue(ca.getActionExistsArgs().live); - } - - //@Test - public void testDestroy1Arg() throws Throwable { - ClientArgs ca = createClientArgs(Arrays.asList( - ACTION_DESTROY, - CLUSTERNAME - )); - assertEquals(CLUSTERNAME, ca.getClusterName()); - assertTrue(ca.getCoreAction() instanceof ActionDestroyArgs); - } - - /** - * Assert that a pass fails with a BadCommandArgumentsException. - * @param argsList - */ - - private void assertParseFails(List argsList) throws SliderException { - try { - ClientArgs clientArgs = createClientArgs(argsList); - Assert.fail("exected an exception, got " + clientArgs); - } catch (BadCommandArgumentsException ignored) { - //expected - } - } - - /** - * Build and parse client args, after adding the base args list. - * @param argsList - */ - public ClientArgs createClientArgs(List argsList) - throws SliderException { - ClientArgs serviceArgs = new ClientArgs(argsList); - serviceArgs.parse(); - return serviceArgs; - } - - public ActionCreateArgs createAction(List argsList) - throws SliderException { - ClientArgs ca = createClientArgs(argsList); - assertEquals(ACTION_CREATE, ca.getAction()); - ActionCreateArgs args = ca.getActionCreateArgs(); - assertNotNull(args); - return args; - } - - //@Test - public void testSingleRoleArg() throws Throwable { - ActionCreateArgs createArgs = createAction(Arrays.asList( - ACTION_CREATE, "cluster1", - ARG_COMPONENT, "master", "5" - )); - List tuples = createArgs.getComponentTuples(); - assertEquals(2, tuples.size()); - Map roleMap = ArgOps.convertTupleListToMap("roles", tuples); - assertEquals("5", roleMap.get("master")); - } - - //@Test - public void testNoRoleArg() throws Throwable { - ActionCreateArgs createArgs = createAction(Arrays.asList( - ACTION_CREATE, "cluster1" - )); - List tuples = createArgs.getComponentTuples(); - Map roleMap = ArgOps.convertTupleListToMap("roles", tuples); - assertNull(roleMap.get("master")); - } - - - //@Test - public void testMultiRoleArgBuild() throws Throwable { - ClientArgs ca = createClientArgs(Arrays.asList( - ACTION_BUILD, "cluster1", - ARG_COMPONENT, "master", "1", - ARG_COMPONENT, "worker", "2" - )); - assertEquals(ACTION_BUILD, ca.getAction()); - assertTrue(ca.getCoreAction() instanceof ActionBuildArgs); - assertTrue(ca.getBuildingActionArgs() instanceof ActionBuildArgs); - AbstractClusterBuildingActionArgs args = ca.getActionBuildArgs(); - List tuples = args.getComponentTuples(); - assertEquals(4, tuples.size()); - Map roleMap = ArgOps.convertTupleListToMap("roles", tuples); - assertEquals("1", roleMap.get("master")); - assertEquals("2", roleMap.get("worker")); - } - - //@Test - public void testArgUpdate() throws Throwable { - ClientArgs ca = createClientArgs(Arrays.asList( - ACTION_UPDATE, "cluster1", - ARG_APPDEF, "app.json" - )); - assertEquals(ACTION_UPDATE, ca.getAction()); - assertTrue(ca.getCoreAction() instanceof ActionUpdateArgs); - assertTrue(ca.getActionUpdateArgs() instanceof ActionUpdateArgs); - AbstractClusterBuildingActionArgs args = ca.getActionUpdateArgs(); - assertNotNull(args.appDef); - } - - //@Test - public void testFlexArgs() throws Throwable { - ClientArgs ca = createClientArgs(Arrays.asList( - ACTION_FLEX, "cluster1", - ARG_COMPONENT, "master", "1", - ARG_COMPONENT, "worker", "2" - )); - assertTrue(ca.getCoreAction() instanceof ActionFlexArgs); - List tuples = ca.getActionFlexArgs().getComponentTuples(); - assertEquals(4, tuples.size()); - Map roleMap = ArgOps.convertTupleListToMap("roles", tuples); - assertEquals("1", roleMap.get("master")); - assertEquals("2", roleMap.get("worker")); - } - - //@Test - public void testDuplicateRole() throws Throwable { - ActionCreateArgs createArgs = createAction(Arrays.asList( - ACTION_CREATE, "cluster1", - ARG_COMPONENT, "master", "1", - ARG_COMPONENT, "master", "2" - )); - List tuples = createArgs.getComponentTuples(); - assertEquals(4, tuples.size()); - try { - Map roleMap = ArgOps.convertTupleListToMap( - "roles", - tuples); - Assert.fail("got a role map $roleMap not a failure"); - } catch (BadCommandArgumentsException expected) { - assertTrue(expected.getMessage().contains(ErrorStrings - .ERROR_DUPLICATE_ENTRY)); - } - } - - //@Test - public void testOddRoleCount() throws Throwable { - ActionCreateArgs createArgs = createAction(Arrays.asList( - ACTION_CREATE, "cluster1", - ARG_COMPONENT, "master", "1", - ARG_COMPONENT, "master", "2" - )); - List tuples = createArgs.getComponentTuples(); - tuples.add("loggers"); - assertEquals(5, tuples.size()); - try { - Map roleMap = ArgOps.convertTupleListToMap("roles", - tuples); - Assert.fail("got a role map " + roleMap + " not a failure"); - } catch (BadCommandArgumentsException expected) { - assertTrue(expected.getMessage().contains(ErrorStrings - .ERROR_PARSE_FAILURE)); - } - } - - /** - * Create some role-opt client args, so that multiple tests can use it. - * @return the args - */ - public ActionCreateArgs createRoleOptClientArgs() throws SliderException { - ActionCreateArgs createArgs = createAction(Arrays.asList( - ACTION_CREATE, "cluster1", - ARG_COMPONENT, "master", "1", - ARG_COMP_OPT, "master", "cheese", "swiss", - ARG_COMP_OPT, "master", "env.CHEESE", "cheddar", - ARG_COMP_OPT, "master", ResourceKeys.YARN_CORES, "3", - - ARG_COMPONENT, "worker", "2", - ARG_COMP_OPT, "worker", ResourceKeys.YARN_CORES, "2", - ARG_COMP_OPT, "worker", RoleKeys.JVM_HEAP, "65536", - ARG_COMP_OPT, "worker", "env.CHEESE", "stilton" - )); - return createArgs; - } - - //@Test - public void testRoleOptionParse() throws Throwable { - ActionCreateArgs createArgs = createRoleOptClientArgs(); - Map> tripleMaps = createArgs.getCompOptionMap(); - Map workerOpts = tripleMaps.get("worker"); - assertEquals(3, workerOpts.size()); - assertEquals("2", workerOpts.get(ResourceKeys.YARN_CORES)); - assertEquals("65536", workerOpts.get(RoleKeys.JVM_HEAP)); - - Map masterOpts = tripleMaps.get("master"); - assertEquals(3, masterOpts.size()); - assertEquals("3", masterOpts.get(ResourceKeys.YARN_CORES)); - - } - - //@Test - public void testRoleOptionsMerge() throws Throwable { - ActionCreateArgs createArgs = createRoleOptClientArgs(); - - Map> roleOpts = createArgs.getCompOptionMap(); - - Map> clusterRoleMap = createEnvMap(); - SliderUtils.applyCommandLineRoleOptsToRoleMap(clusterRoleMap, roleOpts); - - Map masterOpts = clusterRoleMap.get("master"); - assertEquals("swiss", masterOpts.get("cheese")); - - Map workerOpts = clusterRoleMap.get("worker"); - assertEquals("stilton", workerOpts.get("env.CHEESE")); - } - - //@Test - public void testEnvVariableApply() throws Throwable { - ActionCreateArgs createArgs = createRoleOptClientArgs(); - - - Map> roleOpts = createArgs.getCompOptionMap(); - - Map> clusterRoleMap = createEnvMap(); - SliderUtils.applyCommandLineRoleOptsToRoleMap(clusterRoleMap, roleOpts); - - Map workerOpts = clusterRoleMap.get("worker"); - assertEquals("stilton", workerOpts.get("env.CHEESE")); - - Map envmap = SliderUtils.buildEnvMap(workerOpts); - assertEquals("stilton", envmap.get("CHEESE")); - - } - - /** - * Static compiler complaining about matching LinkedHashMap with Map, - * so some explicit creation here. - * @return a map of maps - */ - public Map> createEnvMap() { - - Map cheese = new HashMap<>(); - cheese.put("cheese", "french"); - Map envCheese = new HashMap<>(); - envCheese.put("env.CHEESE", "french"); - Map> envMap = new HashMap<>(); - envMap.put("master", cheese); - envMap.put("worker", envCheese); - return envMap; - } - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestKeytabCommandOptions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestKeytabCommandOptions.java deleted file mode 100644 index 11f8c38..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestKeytabCommandOptions.java +++ /dev/null @@ -1,414 +0,0 @@ -/* - * 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.slider.client; - -import org.apache.commons.io.FileUtils; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RawLocalFileSystem; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.log4j.AppenderSkeleton; -import org.apache.log4j.Logger; -import org.apache.log4j.spi.LoggingEvent; -import org.apache.hadoop.yarn.service.client.params.Arguments; -import org.apache.hadoop.yarn.service.client.params.ClientArgs; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadCommandArgumentsException; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.main.ServiceLauncher; -import org.apache.slider.utils.SliderTestBase; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.UUID; - -/** - * Test a keytab installation. - */ -public class TestKeytabCommandOptions extends SliderTestBase { - - private static SliderFileSystem testFileSystem; - private File testFolderDir; - - @Before - public void setupFilesystem() throws IOException { - org.apache.hadoop.fs.FileSystem fileSystem = new RawLocalFileSystem(); - YarnConfiguration configuration = SliderUtils.createConfiguration(); - fileSystem.setConf(configuration); - testFileSystem = new SliderFileSystem(fileSystem, configuration); - testFolderDir = new File(testFileSystem - .buildKeytabInstallationDirPath("").toUri().getPath()); - FileUtils.deleteDirectory(testFolderDir); - } - - @After - public void cleanup() throws IOException { - if (testFolderDir != null && testFolderDir.exists()) { - FileUtils.deleteDirectory(testFolderDir); - } - } - - //@Test - public void testInstallKeytab() throws Throwable { - // create a mock keytab file - File localKeytab = - FileUtil.createLocalTempFile(getTempLocation(), "test", true); - String contents = UUID.randomUUID().toString(); - FileUtils.write(localKeytab, contents); - YarnConfiguration conf = SliderUtils.createConfiguration(); - ServiceLauncher launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABINSTALL, - ClientArgs.ARG_KEYTAB, - localKeytab.getAbsolutePath(), - Arguments.ARG_FOLDER, - "testFolder")); - Path installedPath = new Path(testFileSystem - .buildKeytabInstallationDirPath("testFolder"), localKeytab.getName()); - File installedKeytab = new File(installedPath.toUri().getPath()); - assertTrue(installedKeytab.exists()); - assertEquals(FileUtils.readFileToString(installedKeytab), - FileUtils.readFileToString(localKeytab)); - } - - //@Test - public void testInstallThenDeleteKeytab() throws Throwable { - // create a mock keytab file - File localKeytab = - FileUtil.createLocalTempFile(getTempLocation(), "test", true); - String contents = UUID.randomUUID().toString(); - FileUtils.write(localKeytab, contents); - YarnConfiguration conf = SliderUtils.createConfiguration(); - ServiceLauncher launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABINSTALL, - ClientArgs.ARG_KEYTAB, - localKeytab.getAbsolutePath(), - Arguments.ARG_FOLDER, - "testFolder")); - Path installedPath = new Path(testFileSystem - .buildKeytabInstallationDirPath("testFolder"), localKeytab.getName()); - File installedKeytab = new File(installedPath.toUri().getPath()); - assertTrue(installedKeytab.exists()); - assertEquals(FileUtils.readFileToString(installedKeytab), - FileUtils.readFileToString(localKeytab)); - - launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABDELETE, - ClientArgs.ARG_KEYTAB, - localKeytab.getName(), - Arguments.ARG_FOLDER, - "testFolder")); - - assertFalse(installedKeytab.exists()); - - } - - //@Test - public void testInstallThenListKeytab() throws Throwable { - // create a mock keytab file - File localKeytab = - FileUtil.createLocalTempFile(getTempLocation(), "test", true); - String contents = UUID.randomUUID().toString(); - FileUtils.write(localKeytab, contents); - YarnConfiguration conf = SliderUtils.createConfiguration(); - ServiceLauncher launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABINSTALL, - ClientArgs.ARG_KEYTAB, - localKeytab.getAbsolutePath(), - Arguments.ARG_FOLDER, - "testFolder")); - Path installedPath = new Path(testFileSystem - .buildKeytabInstallationDirPath("testFolder"), localKeytab.getName()); - File installedKeytab = new File(installedPath.toUri().getPath()); - assertTrue(installedKeytab.exists()); - assertEquals(FileUtils.readFileToString(installedKeytab), - FileUtils.readFileToString(localKeytab)); - - // install an additional copy into another folder to test listing - launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABINSTALL, - ClientArgs.ARG_KEYTAB, - localKeytab.getAbsolutePath(), - Arguments.ARG_FOLDER, - "testFolder2")); - - TestAppender testAppender = new TestAppender(); - - Logger.getLogger(SliderClient.class).addAppender(testAppender); - - try { - launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABLIST) - ); - assertEquals(3, testAppender.events.size()); - String msg = (String) testAppender.events.get(1).getMessage(); - assertTrue(msg.contains("/.slider/keytabs/testFolder")); - assertTrue(msg.endsWith(installedKeytab.getName())); - msg = (String) testAppender.events.get(2).getMessage(); - assertTrue(msg.contains("/.slider/keytabs/testFolder")); - assertTrue(msg.endsWith(installedKeytab.getName())); - } finally { - Logger.getLogger(SliderClient.class).removeAppender(testAppender); - } - - // now listing while specifying the folder name - testAppender = new TestAppender(); - - Logger.getLogger(SliderClient.class).addAppender(testAppender); - - try { - launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABLIST, - Arguments.ARG_FOLDER, - "testFolder")); - assertEquals(2, testAppender.events.size()); - String msg = (String) testAppender.events.get(1).getMessage(); - assertTrue(msg.contains("/.slider/keytabs/testFolder/" + - installedKeytab.getName())); - } finally { - Logger.getLogger(SliderClient.class).removeAppender(testAppender); - } - } - - //@Test - public void testDeleteNonExistentKeytab() throws Throwable { - // create a mock keytab file - YarnConfiguration conf = SliderUtils.createConfiguration(); - try { - ServiceLauncher launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABDELETE, - ClientArgs.ARG_KEYTAB, - "HeyIDontExist.keytab", - Arguments.ARG_FOLDER, - "testFolder")); - fail("expected BadCommandArgumentsException from launch"); - } catch (BadCommandArgumentsException e) { - // expected - } - } - - //@Test - public void testInstallKeytabWithNoFolder() throws Throwable { - // create a mock keytab file - File localKeytab = - FileUtil.createLocalTempFile(getTempLocation(), "test", true); - String contents = UUID.randomUUID().toString(); - FileUtils.write(localKeytab, contents); - YarnConfiguration conf = SliderUtils.createConfiguration(); - try { - ServiceLauncher launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABINSTALL, - ClientArgs.ARG_KEYTAB, - localKeytab.getAbsolutePath())); - fail("expected BadCommandArgumentsException from launch"); - } catch (BadCommandArgumentsException e) { - // expected - } - } - - //@Test - public void testInstallKeytabWithNoKeytab() throws Throwable { - // create a mock keytab file - File localKeytab = - FileUtil.createLocalTempFile(getTempLocation(), "test", true); - String contents = UUID.randomUUID().toString(); - FileUtils.write(localKeytab, contents); - YarnConfiguration conf = SliderUtils.createConfiguration(); - try { - ServiceLauncher launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABINSTALL, - ClientArgs.ARG_FOLDER, - "testFolder")); - fail("expected BadCommandArgumentsException from launch"); - } catch (BadCommandArgumentsException e) { - // expected - } - } - - //@Test - public void testInstallKeytabAllowingOverwrite() throws Throwable { - // create a mock keytab file - File localKeytab = - FileUtil.createLocalTempFile(getTempLocation(), "test", true); - String contents = UUID.randomUUID().toString(); - FileUtils.write(localKeytab, contents); - YarnConfiguration conf = SliderUtils.createConfiguration(); - ServiceLauncher launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABINSTALL, - ClientArgs.ARG_KEYTAB, - localKeytab.getAbsolutePath(), - Arguments.ARG_FOLDER, - "testFolder")); - Path installedPath = new Path(testFileSystem - .buildKeytabInstallationDirPath("testFolder"), localKeytab.getName()); - File installedKeytab = new File(installedPath.toUri().getPath()); - assertTrue(installedKeytab.exists()); - assertEquals(FileUtils.readFileToString(installedKeytab), FileUtils - .readFileToString(localKeytab)); - launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABINSTALL, - ClientArgs.ARG_KEYTAB, - localKeytab.getAbsolutePath(), - Arguments.ARG_FOLDER, - "testFolder", - Arguments.ARG_OVERWRITE) - ); - assertTrue(installedKeytab.exists()); - assertEquals(FileUtils.readFileToString(installedKeytab), - FileUtils.readFileToString(localKeytab)); - } - - //@Test - public void testInstallKeytabNotAllowingOverwrite() throws Throwable { - // create a mock keytab file - File localKeytab = - FileUtil.createLocalTempFile(getTempLocation(), "test", true); - String contents = UUID.randomUUID().toString(); - FileUtils.write(localKeytab, contents); - YarnConfiguration conf = SliderUtils.createConfiguration(); - ServiceLauncher launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABINSTALL, - ClientArgs.ARG_KEYTAB, - localKeytab.getAbsolutePath(), - Arguments.ARG_FOLDER, - "testFolder")); - Path installedPath = new Path(testFileSystem - .buildKeytabInstallationDirPath("testFolder"), localKeytab.getName()); - File installedKeytab = new File(installedPath.toUri().getPath()); - assertTrue(installedKeytab.exists()); - assertEquals(FileUtils.readFileToString(installedKeytab), - FileUtils.readFileToString(localKeytab)); - try { - launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABINSTALL, - ClientArgs.ARG_KEYTAB, - localKeytab.getAbsolutePath(), - Arguments.ARG_FOLDER, - "testFolder")); - fail("expected BadCommandArgumentsException from launch"); - } catch (BadCommandArgumentsException e) { - // expected - } - } - - //@Test - public void testInstallKeytabWithMissingKeytab() throws Throwable { - // create a mock keytab file - YarnConfiguration conf = SliderUtils.createConfiguration(); - try { - ServiceLauncher launcher = launch(TestSliderClient.class, - conf, - Arrays.asList( - ClientArgs.ACTION_KEYTAB, - ClientArgs.ARG_KEYTABINSTALL, - ClientArgs.ARG_KEYTAB, - "HeyIDontExist.keytab", - Arguments.ARG_FOLDER, - "testFolder")); - fail("expected BadCommandArgumentsException from launch"); - } catch (BadCommandArgumentsException e) { - // expected - } - } - - private File getTempLocation() { - return new File(System.getProperty("user.dir") + "/target"); - } - - /** - * Test SliderClient with overridden filesystem. - */ - public static class TestSliderClient extends SliderClient { - public TestSliderClient() { - super(); - } - - @Override - protected void initHadoopBinding() throws IOException, SliderException { - sliderFileSystem = testFileSystem; - } - - } - - /** - * Appender that captures logging events. - */ - public static class TestAppender extends AppenderSkeleton { - private List events = new ArrayList<>(); - - public void close() {} - - public boolean requiresLayout() { - return false; - } - - @Override - protected void append(LoggingEvent event) { - events.add(event); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestSliderClientMethods.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestSliderClientMethods.java deleted file mode 100644 index 66aa7b8..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestSliderClientMethods.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * 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.slider.client; - -import org.apache.hadoop.util.Shell; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.server.appmaster.model.mock.MockApplicationId; -import org.apache.slider.utils.SliderTestBase; -import org.easymock.EasyMock; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.powermock.api.easymock.PowerMock; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Map; - -/** - * Test slider client methods. - */ -@RunWith(PowerMockRunner.class) -@PrepareForTest(SliderUtils.class) -public class TestSliderClientMethods extends SliderTestBase { - protected static final Logger LOG = - LoggerFactory.getLogger(TestSliderClientMethods.class); - - static final String AM_ENV = "LD_LIBRARY_PATH"; - static final String PLACEHOLDER_KEY = "${distro.version}"; - static final String PLACEHOLDER_SYSTEM_KEY = "DISTRO_VERSION"; - static final String PLACEHOLDER_VALUE = "1.0.0"; - static final String AM_ENV_2 = "PATH"; - static final String PLACEHOLDER_KEY_2 = "${native.version}"; - static final String PLACEHOLDER_SYSTEM_KEY_2 = "NATIVE_VERSION"; - static final String PLACEHOLDER_VALUE_2 = "2.0.0"; - - //@Test - public void testGeneratePlaceholderKeyValueMap() throws Throwable { - TestSliderClient testSliderClient = new TestSliderClient(); - - PowerMock.mockStatic(System.class); - EasyMock.expect(SliderUtils.getSystemEnv(PLACEHOLDER_SYSTEM_KEY)) - .andReturn(PLACEHOLDER_VALUE).anyTimes(); - PowerMock.replayAll(); - - Map placeholders = testSliderClient - .generatePlaceholderKeyValueMap(AM_ENV + "=/usr/lib/" + - PLACEHOLDER_KEY); - Assert.assertTrue(placeholders.containsKey(PLACEHOLDER_KEY)); - Assert.assertEquals("Should be equal", PLACEHOLDER_VALUE, - placeholders.get(PLACEHOLDER_KEY)); - - PowerMock.verifyAll(); - LOG.info("Placeholders = {}", placeholders); - } - - //@Test - public void testSetAmLaunchEnv() throws Throwable { - TestSliderClient testSliderClient = new TestSliderClient(); - YarnConfiguration conf = SliderUtils.createConfiguration(); - conf.set(SliderXmlConfKeys.KEY_AM_LAUNCH_ENV, AM_ENV + "=/usr/lib/" - + PLACEHOLDER_KEY); - - PowerMock.mockStatic(System.class); - EasyMock.expect(SliderUtils.getSystemEnv(PLACEHOLDER_SYSTEM_KEY)) - .andReturn(PLACEHOLDER_VALUE); - PowerMock.replayAll(); - - Map amLaunchEnv = testSliderClient.getAmLaunchEnv(conf); - Assert.assertNotNull(amLaunchEnv); - Assert.assertNotNull(amLaunchEnv.get(AM_ENV)); - Assert.assertEquals("Should be equal", amLaunchEnv.get(AM_ENV), - (Shell.WINDOWS ? "%" + AM_ENV + "%;" : "$" + AM_ENV + ":") + - "/usr/lib/" + PLACEHOLDER_VALUE); - - PowerMock.verifyAll(); - LOG.info("amLaunchEnv = {}", amLaunchEnv); - } - - //@Test - public void testSetAmLaunchEnvMulti() throws Throwable { - TestSliderClient testSliderClient = new TestSliderClient(); - YarnConfiguration conf = SliderUtils.createConfiguration(); - conf.set(SliderXmlConfKeys.KEY_AM_LAUNCH_ENV, AM_ENV + "=/usr/lib/" - + PLACEHOLDER_KEY + "," + AM_ENV_2 + "=/usr/bin/" + PLACEHOLDER_KEY_2); - - PowerMock.mockStatic(System.class); - EasyMock.expect(SliderUtils.getSystemEnv(PLACEHOLDER_SYSTEM_KEY)) - .andReturn(PLACEHOLDER_VALUE); - EasyMock.expect(SliderUtils.getSystemEnv(PLACEHOLDER_SYSTEM_KEY_2)) - .andReturn(PLACEHOLDER_VALUE_2); - PowerMock.replayAll(); - - Map amLaunchEnv = testSliderClient.getAmLaunchEnv(conf); - Assert.assertNotNull(amLaunchEnv); - Assert.assertEquals("Should have 2 envs", amLaunchEnv.size(), 2); - Assert.assertNotNull(amLaunchEnv.get(AM_ENV)); - Assert.assertEquals("Should be equal", amLaunchEnv.get(AM_ENV), - (Shell.WINDOWS ? "%" + AM_ENV + "%;" : "$" + AM_ENV + ":") + - "/usr/lib/" + PLACEHOLDER_VALUE); - Assert.assertNotNull(amLaunchEnv.get(AM_ENV_2)); - Assert.assertEquals("Should be equal", amLaunchEnv.get(AM_ENV_2), - (Shell.WINDOWS ? "%" + AM_ENV_2 + "%;" : "$" + AM_ENV_2 + ":") + - "/usr/bin/" + PLACEHOLDER_VALUE_2); - - PowerMock.verifyAll(); - LOG.info("amLaunchEnv = " + amLaunchEnv); - } - - static class TestSliderClient extends SliderClient { - @Override - public ApplicationId submitApplication(ApplicationSubmissionContext - context) - throws YarnException, IOException { - return new MockApplicationId(1); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestSliderTokensCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestSliderTokensCommand.java deleted file mode 100644 index d140521..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/client/TestSliderTokensCommand.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * 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.slider.client; - -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.slider.common.params.ActionTokensArgs; -import org.apache.hadoop.yarn.service.client.params.Arguments; -import org.apache.hadoop.yarn.service.client.params.SliderActions; -import org.apache.slider.core.exceptions.BadClusterStateException; -import org.apache.slider.core.exceptions.NotFoundException; -import org.apache.slider.utils.SliderTestBase; -import org.junit.Test; - -import java.util.Arrays; - -/** - * Test the argument parsing/validation logic. - */ -public class TestSliderTokensCommand extends SliderTestBase { - - private static YarnConfiguration config = createTestConfig(); - - public static YarnConfiguration createTestConfig() { - YarnConfiguration configuration = new YarnConfiguration(); - configuration.set(YarnConfiguration.RM_ADDRESS, "127.0.0.1:8032"); - return configuration; - } - - //@Test - public void testBadSourceArgs() throws Throwable { - launchExpectingException(SliderClient.class, - config, - ActionTokensArgs.DUPLICATE_ARGS, - Arrays.asList(SliderActions.ACTION_TOKENS, - Arguments.ARG_SOURCE, "target/tokens.bin", - Arguments.ARG_OUTPUT, "target/tokens.bin" - )); - } - - //@Test - public void testKTNoPrincipal() throws Throwable { - launchExpectingException(SliderClient.class, - config, - ActionTokensArgs.MISSING_KT_PROVIDER, - Arrays.asList(SliderActions.ACTION_TOKENS, - Arguments.ARG_KEYTAB, "target/keytab" - )); - } - - //@Test - public void testPrincipalNoKT() throws Throwable { - launchExpectingException(SliderClient.class, - config, - ActionTokensArgs.MISSING_KT_PROVIDER, - Arrays.asList(SliderActions.ACTION_TOKENS, - Arguments.ARG_PRINCIPAL, "bob@REALM" - )); - } - - /** - * A missing keytab is an error. - * @throws Throwable - */ - //@Test - public void testMissingKT() throws Throwable { - Throwable ex = launchExpectingException(SliderClient.class, - config, - TokensOperation.E_NO_KEYTAB, - Arrays.asList(SliderActions.ACTION_TOKENS, - Arguments.ARG_PRINCIPAL, "bob@REALM", - Arguments.ARG_KEYTAB, "target/keytab" - )); - if (!(ex instanceof NotFoundException)) { - throw ex; - } - } - - //@Test - public void testMissingSourceFile() throws Throwable { - Throwable ex = launchExpectingException(SliderClient.class, - config, - TokensOperation.E_MISSING_SOURCE_FILE, - Arrays.asList(SliderActions.ACTION_TOKENS, - Arguments.ARG_SOURCE, "target/tokens.bin" - )); - if (!(ex instanceof NotFoundException)) { - throw ex; - } - } - - //@Test - public void testListHarmlessWhenInsecure() throws Throwable { - execSliderCommand(0, config, Arrays.asList(SliderActions.ACTION_TOKENS)); - } - - //@Test - public void testCreateFailsWhenInsecure() throws Throwable { - Throwable ex = launchExpectingException(SliderClient.class, - config, - TokensOperation.E_INSECURE, - Arrays.asList(SliderActions.ACTION_TOKENS, - Arguments.ARG_OUTPUT, "target/tokens.bin" - )); - if (!(ex instanceof BadClusterStateException)) { - throw ex; - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestConfigHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestConfigHelper.java deleted file mode 100644 index b452aba..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestConfigHelper.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.slider.common.tools; - -import org.apache.hadoop.conf.Configuration; -import org.apache.slider.utils.YarnMiniClusterTestBase; -import org.junit.Test; - -import java.io.ByteArrayInputStream; -import java.io.InputStream; -import java.util.Map; - -/** - * Test config helper. - */ -public class TestConfigHelper extends YarnMiniClusterTestBase { - - //@Test - public void testConfigLoaderIteration() throws Throwable { - - String xml = "key" + - "valueprogramatically" + - ""; - InputStream ins = new ByteArrayInputStream(xml.getBytes("UTF8")); - Configuration conf = new Configuration(false); - conf.addResource(ins); - Configuration conf2 = new Configuration(false); - for (Map.Entry entry : conf) { - conf2.set(entry.getKey(), entry.getValue(), "src"); - } - - } - - //@Test - public void testConfigDeprecation() throws Throwable { - ConfigHelper.registerDeprecatedConfigItems(); - Configuration conf = new Configuration(false); - // test deprecated items here - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestConfigHelperHDFS.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestConfigHelperHDFS.java deleted file mode 100644 index 1853c84..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestConfigHelperHDFS.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.slider.common.tools; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.slider.utils.YarnMiniClusterTestBase; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.net.URI; - -/** - * Test config helper loading configuration from HDFS. - */ -public class TestConfigHelperHDFS extends YarnMiniClusterTestBase { - private static final Logger LOG = - LoggerFactory.getLogger(TestConfigHelperHDFS.class); - - //@Test - public void testConfigHelperHDFS() throws Throwable { - YarnConfiguration config = getConfiguration(); - createMiniHDFSCluster("testConfigHelperHDFS", config); - - Configuration conf = new Configuration(false); - conf.set("key", "value"); - URI fsURI = new URI(getFsDefaultName()); - Path root = new Path(fsURI); - Path confPath = new Path(root, "conf.xml"); - FileSystem dfs = FileSystem.get(fsURI, config); - ConfigHelper.saveConfig(dfs, confPath, conf); - //load time - Configuration loaded = ConfigHelper.loadConfiguration(dfs, confPath); - LOG.info(ConfigHelper.dumpConfigToString(loaded)); - assertEquals("value", loaded.get("key")); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestExecutionEnvironment.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestExecutionEnvironment.java deleted file mode 100644 index f08bf31..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestExecutionEnvironment.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.slider.common.tools; - -import org.apache.slider.utils.SliderTestBase; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Test execution environment. - */ -public class TestExecutionEnvironment extends SliderTestBase { - protected static final Logger LOG = - LoggerFactory.getLogger(TestExecutionEnvironment.class); - - //@Test - public void testClientEnv() throws Throwable { - SliderUtils.validateSliderClientEnvironment(LOG); - } - - //@Test - public void testWinutils() throws Throwable { - SliderUtils.maybeVerifyWinUtilsValid(); - } - - //@Test - public void testServerEnv() throws Throwable { - SliderUtils.validateSliderServerEnvironment(LOG, true); - } - - //@Test - public void testServerEnvNoDependencies() throws Throwable { - SliderUtils.validateSliderServerEnvironment(LOG, false); - } - - //@Test - public void testopenSSLEnv() throws Throwable { - SliderUtils.validateOpenSSLEnv(LOG); - } - - //@Test - public void testValidatePythonEnv() throws Throwable { - SliderUtils.validatePythonEnv(LOG); - } - - //@Test - public void testNativeLibs() throws Throwable { - assertNativeLibrariesPresent(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestPortScan.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestPortScan.java deleted file mode 100644 index a161779..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestPortScan.java +++ /dev/null @@ -1,184 +0,0 @@ -/* - * 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.slider.common.tools; - -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.core.exceptions.SliderException; -import org.junit.Test; - -import java.net.ServerSocket; -import java.util.Arrays; -import java.util.List; - -import static org.junit.Assert.*; - -/** - * Test finding a port in a range. - */ -public class TestPortScan { - - //@Test - public void testScanPorts() throws Throwable { - - ServerSocket server = new ServerSocket(0); - - try { - int serverPort = server.getLocalPort(); - assertFalse(SliderUtils.isPortAvailable(serverPort)); - int port = SliderUtils.findFreePort(serverPort, 10); - assertTrue(port > 0 && serverPort < port); - } finally { - server.close(); - } - } - - //@Test - public void testRequestedPortsLogic() throws Throwable { - PortScanner portScanner = new PortScanner(); - portScanner.setPortRange("5,6,8-10, 11,14 ,20 - 22"); - List ports = portScanner.getRemainingPortsToCheck(); - List expectedPorts = - Arrays.asList(5, 6, 8, 9, 10, 11, 14, 20, 21, 22); - assertEquals(expectedPorts, ports); - } - - //@Test - public void testRequestedPortsOutOfOrder() throws Throwable { - PortScanner portScanner = new PortScanner(); - portScanner.setPortRange("8-10,5,6, 11,20 - 22, 14 "); - List ports = portScanner.getRemainingPortsToCheck(); - List expectedPorts = - Arrays.asList(5, 6, 8, 9, 10, 11, 14, 20, 21, 22); - assertEquals(expectedPorts, ports); - } - - //@Test - public void testFindAvailablePortInRange() throws Throwable { - ServerSocket server = new ServerSocket(0); - try { - int serverPort = server.getLocalPort(); - - PortScanner portScanner = new PortScanner(); - portScanner.setPortRange("" + (serverPort-1) + "-" + (serverPort + 3)); - int port = portScanner.getAvailablePort(); - assertNotEquals(port, serverPort); - assertTrue(port >= serverPort -1 && port <= serverPort + 3); - } finally { - server.close(); - } - } - - //@Test - public void testFindAvailablePortInList() throws Throwable { - ServerSocket server = new ServerSocket(0); - try { - int serverPort = server.getLocalPort(); - - PortScanner portScanner = new PortScanner(); - portScanner.setPortRange("" + (serverPort-1) + ", " + (serverPort + 1)); - int port = portScanner.getAvailablePort(); - assertNotEquals(port, serverPort); - assertTrue(port == serverPort -1 || port == serverPort + 1); - } finally { - server.close(); - } - } - - //@Test - public void testNoAvailablePorts() throws Throwable { - ServerSocket server1 = new ServerSocket(0); - ServerSocket server2 = new ServerSocket(0); - try { - int serverPort1 = server1.getLocalPort(); - int serverPort2 = server2.getLocalPort(); - - PortScanner portScanner = new PortScanner(); - portScanner.setPortRange("" + serverPort1+ ", " + serverPort2); - try { - portScanner.getAvailablePort(); - fail("expected SliderException"); - } catch (SliderException e) { - // expected - } - } finally { - server1.close(); - server2.close(); - } - } - - //@Test - public void testPortRemovedFromRange() throws Throwable { - ServerSocket server = new ServerSocket(0); - try { - int serverPort = server.getLocalPort(); - - PortScanner portScanner = new PortScanner(); - portScanner.setPortRange("" + (serverPort-1) + "-" + (serverPort + 3)); - int port = portScanner.getAvailablePort(); - assertNotEquals(port, serverPort); - assertTrue(port >= serverPort -1 && port <= serverPort + 3); - assertFalse(portScanner.getRemainingPortsToCheck().contains(port)); - } finally { - server.close(); - } - } - - //@Test(expected = BadConfigException.class) - public void testBadRange() throws BadConfigException { - PortScanner portScanner = new PortScanner(); - // note the em dash - portScanner.setPortRange("2000–2010"); - } - - //@Test(expected = BadConfigException.class) - public void testEndBeforeStart() throws BadConfigException { - PortScanner portScanner = new PortScanner(); - portScanner.setPortRange("2001-2000"); - } - - //@Test(expected = BadConfigException.class) - public void testEmptyRange() throws BadConfigException { - PortScanner portScanner = new PortScanner(); - portScanner.setPortRange(""); - } - - //@Test(expected = BadConfigException.class) - public void testBlankRange() throws BadConfigException { - PortScanner portScanner = new PortScanner(); - portScanner.setPortRange(" "); - } - - //@Test - public void testExtraComma() throws BadConfigException { - PortScanner portScanner = new PortScanner(); - portScanner.setPortRange("2000-2001, "); - List ports = portScanner.getRemainingPortsToCheck(); - List expectedPorts = Arrays.asList(2000, 2001); - assertEquals(expectedPorts, ports); - } - - //@Test - public void testExtraCommas() throws BadConfigException { - PortScanner portScanner = new PortScanner(); - portScanner.setPortRange("2000-2001,, ,2003,"); - List ports = portScanner.getRemainingPortsToCheck(); - List expectedPorts = Arrays.asList(2000, 2001, 2003); - assertEquals(expectedPorts, ports); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderFileSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderFileSystem.java deleted file mode 100644 index 755a4c6..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderFileSystem.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.slider.common.tools; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; -import org.apache.slider.utils.SliderTestBase; -import org.junit.Test; - -/** - * Test slider file system. - */ -public class TestSliderFileSystem extends SliderTestBase { - private static Configuration defaultConfiguration() { - return new Configuration(); - } - - private static Configuration createConfigurationWithKV(String key, String - value) { - Configuration conf = defaultConfiguration(); - conf.set(key, value); - return conf; - } - - //@Test - public void testSliderBasePathDefaultValue() throws Throwable { - Configuration configuration = defaultConfiguration(); - FileSystem fileSystem = FileSystem.get(configuration); - - SliderFileSystem fs2 = new SliderFileSystem(fileSystem, configuration); - assertEquals(fs2.getBaseApplicationPath(), new Path(fileSystem - .getHomeDirectory(), ".slider")); - } - - //@Test - public void testSliderBasePathCustomValue() throws Throwable { - Configuration configuration = createConfigurationWithKV(SliderXmlConfKeys - .KEY_SLIDER_BASE_PATH, "/slider/cluster"); - FileSystem fileSystem = FileSystem.get(configuration); - SliderFileSystem fs2 = new SliderFileSystem(fileSystem, configuration); - - assertEquals(fs2.getBaseApplicationPath(), new Path("/slider/cluster")); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderTestUtils.java deleted file mode 100644 index e19e33f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderTestUtils.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * 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.slider.common.tools; - -import org.apache.hadoop.conf.Configuration; -import org.apache.slider.utils.SliderTestUtils; -import org.junit.Test; -import org.junit.internal.AssumptionViolatedException; - -/** - * Test slider test utils. - */ -public class TestSliderTestUtils extends SliderTestUtils { - - //@Test - public void testAssumeTrue() throws Throwable { - - try { - assume(true, "true"); - } catch (AssumptionViolatedException e) { - throw new Exception(e); - } - } - - //@Test - public void testAssumeFalse() throws Throwable { - - try { - assume(false, "false"); - fail("expected an exception"); - } catch (AssumptionViolatedException ignored) { - //expected - } - } - - //@Test - public void testAssumeBoolOptionSetInConf() throws Throwable { - Configuration conf = new Configuration(false); - conf.set("key", "true"); - try { - assumeBoolOption(conf, "key", false); - } catch (AssumptionViolatedException e) { - throw new Exception(e); - } - } - - //@Test - public void testAssumeBoolOptionUnsetInConf() throws Throwable { - Configuration conf = new Configuration(false); - try { - assumeBoolOption(conf, "key", true); - } catch (AssumptionViolatedException e) { - throw new Exception(e); - } - } - - - //@Test - public void testAssumeBoolOptionFalseInConf() throws Throwable { - Configuration conf = new Configuration(false); - conf.set("key", "false"); - try { - assumeBoolOption(conf, "key", true); - fail("expected an exception"); - } catch (AssumptionViolatedException ignored) { - //expected - } - } - - //@Test - public void testAssumeBoolOptionFalseUnsetInConf() throws Throwable { - Configuration conf = new Configuration(false); - try { - assumeBoolOption(conf, "key", false); - fail("expected an exception"); - } catch (AssumptionViolatedException ignored) { - //expected - } - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java deleted file mode 100644 index 057f6c5..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * 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.slider.common.tools; - -import org.apache.commons.io.FileUtils; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationReportPBImpl; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -/** Test slider util methods. */ -public class TestSliderUtils { - protected static final Logger log = - LoggerFactory.getLogger(TestSliderUtils.class); - @Rule - public TemporaryFolder folder = new TemporaryFolder(); - - //@Test - public void testTruncate() { - Assert.assertEquals(SliderUtils.truncate(null, 5), null); - Assert.assertEquals(SliderUtils.truncate("323", -1), "323"); - Assert.assertEquals(SliderUtils.truncate("3232", 5), "3232"); - Assert.assertEquals(SliderUtils.truncate("1234567890", 0), "1234567890"); - Assert.assertEquals(SliderUtils.truncate("123456789012345", 15), "123456789012345"); - Assert.assertEquals(SliderUtils.truncate("123456789012345", 14), "12345678901..."); - Assert.assertEquals(SliderUtils.truncate("1234567890", 1), "1"); - Assert.assertEquals(SliderUtils.truncate("1234567890", 10), "1234567890"); - Assert.assertEquals(SliderUtils.truncate("", 10), ""); - } - - //@Test - public void testApplicationReportComparison() { - List instances = getApplicationReports(); - - SliderUtils.sortApplicationsByMostRecent(instances); - - Assert.assertEquals(1000, instances.get(0).getStartTime()); - Assert.assertEquals(1000, instances.get(1).getStartTime()); - Assert.assertEquals(1000, instances.get(2).getStartTime()); - Assert.assertEquals(1000, instances.get(3).getStartTime()); - - instances = getApplicationReports(); - - SliderUtils.sortApplicationReport(instances); - Assert.assertEquals(1000, instances.get(0).getStartTime()); - Assert.assertEquals(1000, instances.get(1).getStartTime()); - Assert.assertEquals(1000, instances.get(2).getStartTime()); - Assert.assertEquals(1000, instances.get(3).getStartTime()); - - Assert.assertTrue(instances.get(0).getYarnApplicationState() == YarnApplicationState.ACCEPTED || - instances.get(0).getYarnApplicationState() == YarnApplicationState.RUNNING); - Assert.assertTrue(instances.get(1).getYarnApplicationState() == YarnApplicationState.ACCEPTED || - instances.get(1).getYarnApplicationState() == YarnApplicationState.RUNNING); - Assert.assertTrue(instances.get(2).getYarnApplicationState() == YarnApplicationState.ACCEPTED || - instances.get(2).getYarnApplicationState() == YarnApplicationState.RUNNING); - Assert.assertTrue(instances.get(3).getYarnApplicationState() == YarnApplicationState.KILLED); - } - - private List getApplicationReports() { - List instances = new ArrayList(); - instances.add(getApplicationReport(1000, 0, "app1", YarnApplicationState.ACCEPTED)); - instances.add(getApplicationReport(900, 998, "app1", YarnApplicationState.KILLED)); - instances.add(getApplicationReport(900, 998, "app2", YarnApplicationState.FAILED)); - instances.add(getApplicationReport(1000, 0, "app2", YarnApplicationState.RUNNING)); - instances.add(getApplicationReport(800, 837, "app3", YarnApplicationState.FINISHED)); - instances.add(getApplicationReport(1000, 0, "app3", YarnApplicationState.RUNNING)); - instances.add(getApplicationReport(900, 998, "app3", YarnApplicationState.KILLED)); - instances.add(getApplicationReport(800, 837, "app4", YarnApplicationState.FINISHED)); - instances.add(getApplicationReport(1000, 1050, "app4", YarnApplicationState.KILLED)); - instances.add(getApplicationReport(900, 998, "app4", YarnApplicationState.FINISHED)); - - Assert.assertEquals("app1", instances.get(0).getApplicationType()); - Assert.assertEquals("app1", instances.get(1).getApplicationType()); - Assert.assertEquals("app2", instances.get(2).getApplicationType()); - Assert.assertEquals("app2", instances.get(3).getApplicationType()); - return instances; - } - - private ApplicationReportPBImpl getApplicationReport(long startTime, - long finishTime, - String name, - YarnApplicationState state) { - ApplicationReportPBImpl ar = new ApplicationReportPBImpl(); - ar.setFinishTime(finishTime); - ar.setStartTime(startTime); - ar.setApplicationType(name); - ar.setYarnApplicationState(state); - return ar; - } - - - //@Test - public void testGetHdpVersion() { - String hdpVersion = "2.3.2.0-2766"; - Assert.assertEquals("Version should be empty", null, - SliderUtils.getHdpVersion()); - } - - //@Test - public void testIsHdp() { - Assert.assertFalse("Should be false", SliderUtils.isHdp()); - } - - //@Test - public void testWrite() throws IOException { - File testWriteFile = folder.newFile("testWrite"); - SliderUtils.write(testWriteFile, "test".getBytes("UTF-8")); - Assert.assertTrue(FileUtils.readFileToString(testWriteFile, "UTF-8").equals("test")); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestWindowsSupport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestWindowsSupport.java deleted file mode 100644 index 073fd51..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestWindowsSupport.java +++ /dev/null @@ -1,177 +0,0 @@ -/* - * 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.slider.common.tools; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.ChecksumFileSystem; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.service.ServiceStateException; -import org.apache.hadoop.util.Shell; -import org.apache.slider.utils.YarnMiniClusterTestBase; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.FileNotFoundException; -import java.net.URI; -import java.util.Arrays; -import java.util.List; -import java.util.regex.Pattern; - -/** - * Test windows support. - */ -public class TestWindowsSupport extends YarnMiniClusterTestBase { - private static final Logger LOG = - LoggerFactory.getLogger(TestWindowsSupport.class); - - private static final Pattern HAS_DRIVE_LETTER_SPECIFIER = - Pattern.compile("^/?[a-zA-Z]:"); - public static final String WINDOWS_FILE = - "C:\\Users\\Administrator\\AppData\\Local\\Temp" + - "\\junit3180177850133852404\\testpkg\\appdef_1.zip"; - - - private static boolean hasWindowsDrive(String path) { - return HAS_DRIVE_LETTER_SPECIFIER.matcher(path).find(); - } - - private static int startPositionWithoutWindowsDrive(String path) { - if (hasWindowsDrive(path)) { - return path.charAt(0) == '/' ? 3 : 2; - } else { - return 0; - } - } - - //@Test - public void testHasWindowsDrive() throws Throwable { - assertTrue(hasWindowsDrive(WINDOWS_FILE)); - } - - //@Test - public void testStartPosition() throws Throwable { - assertEquals(2, startPositionWithoutWindowsDrive(WINDOWS_FILE)); - } - - //@Test - public void testPathHandling() throws Throwable { - assumeWindows(); - - Path path = new Path(WINDOWS_FILE); - URI uri = path.toUri(); - // assert "file" == uri.scheme - assertNull(uri.getAuthority()); - - Configuration conf = new Configuration(); - - FileSystem localfs = FileSystem.get(uri, conf); - assertTrue(localfs instanceof ChecksumFileSystem); - try { - FileStatus stat = localfs.getFileStatus(path); - fail("expected an exception, got " + stat); - } catch (FileNotFoundException fnfe) { - // expected - } - - try { - FSDataInputStream appStream = localfs.open(path); - } catch (FileNotFoundException fnfe) { - // expected - } - } - - //@Test - public void testExecNonexistentBinary() throws Throwable { - assumeWindows(); - List commands = Arrays.asList("undefined-application", "--version"); - try { - exec(0, commands); - fail("expected an exception"); - } catch (ServiceStateException e) { - if (!(e.getCause() instanceof FileNotFoundException)) { - throw e; - } - } - } - //@Test - public void testExecNonexistentBinary2() throws Throwable { - assumeWindows(); - assertFalse(doesAppExist(Arrays.asList("undefined-application", - "--version"))); - } - - //@Test - public void testEmitKillCommand() throws Throwable { - - int result = killJavaProcesses("regionserver", 9); - // we know the exit code if there is no supported kill operation - assertTrue(getKillSupported() || result == -1); - } - - //@Test - public void testHadoopHomeDefined() throws Throwable { - assumeWindows(); - String hadoopHome = Shell.getHadoopHome(); - LOG.info("HADOOP_HOME={}", hadoopHome); - } - - //@Test - public void testHasWinutils() throws Throwable { - assumeWindows(); - SliderUtils.maybeVerifyWinUtilsValid(); - } - - //@Test - public void testExecWinutils() throws Throwable { - assumeWindows(); - String winUtilsPath = Shell.getWinUtilsPath(); - assertTrue(SliderUtils.isSet(winUtilsPath)); - File winUtils = new File(winUtilsPath); - LOG.debug("Winutils is at {}", winUtils); - - exec(0, Arrays.asList(winUtilsPath, "systeminfo")); - } - - //@Test - public void testPath() throws Throwable { - String path = extractPath(); - LOG.info("Path value = {}", path); - } - - //@Test - public void testFindJavac() throws Throwable { - String name = Shell.WINDOWS ? "javac.exe" : "javac"; - assertNotNull(locateExecutable(name)); - } - - //@Test - public void testHadoopDLL() throws Throwable { - assumeWindows(); - // split the path - File exepath = locateExecutable("HADOOP.DLL"); - assertNotNull(exepath); - LOG.info("Hadoop DLL at: {}", exepath); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestZKIntegration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestZKIntegration.java deleted file mode 100644 index ed9337d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestZKIntegration.java +++ /dev/null @@ -1,187 +0,0 @@ -/* - * 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.slider.common.tools; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.registry.server.services.MicroZookeeperServiceKeys; -import org.apache.slider.client.SliderClient; -import org.apache.slider.core.zk.ZKIntegration; -import org.apache.slider.utils.KeysForTests; -import org.apache.slider.utils.YarnZKMiniClusterTestBase; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooDefs; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.util.List; - -/** - * Test ZK integration. - */ -public class TestZKIntegration extends YarnZKMiniClusterTestBase implements - KeysForTests { - private static final Logger LOG = - LoggerFactory.getLogger(TestZKIntegration.class); - - public static final String USER = KeysForTests.USERNAME; - public static final int CONNECT_TIMEOUT = 5000; - private ZKIntegration zki; - - @Before - public void createCluster() { - Configuration conf = getConfiguration(); - String name = methodName.getMethodName(); - File zkdir = new File("target/zk/${name}"); - FileUtil.fullyDelete(zkdir); - conf.set(MicroZookeeperServiceKeys.KEY_ZKSERVICE_DIR, zkdir - .getAbsolutePath()); - createMicroZKCluster("-"+ name, conf); - } - - @After - public void closeZKI() throws IOException { - if (zki != null) { - zki.close(); - zki = null; - } - } - - public ZKIntegration initZKI() throws IOException, InterruptedException { - zki = createZKIntegrationInstance( - getZKBinding(), methodName.getMethodName(), true, false, - CONNECT_TIMEOUT); - return zki; - } - - //@Test - public void testListUserClustersWithoutAnyClusters() throws Throwable { - assertHasZKCluster(); - initZKI(); - String userPath = ZKIntegration.mkSliderUserPath(USER); - List clusters = this.zki.getClusters(); - assertTrue(SliderUtils.isEmpty(clusters)); - } - - //@Test - public void testListUserClustersWithOneCluster() throws Throwable { - assertHasZKCluster(); - - initZKI(); - String userPath = ZKIntegration.mkSliderUserPath(USER); - String fullPath = zki.createPath(userPath, "/cluster-", - ZooDefs.Ids.OPEN_ACL_UNSAFE, - CreateMode.EPHEMERAL_SEQUENTIAL); - LOG.info("Ephemeral path {}", fullPath); - List clusters = zki.getClusters(); - assertEquals(1, clusters.size()); - assertTrue(fullPath.endsWith(clusters.get(0))); - } - - //@Test - public void testListUserClustersWithTwoCluster() throws Throwable { - initZKI(); - String userPath = ZKIntegration.mkSliderUserPath(USER); - String c1 = createEphemeralChild(zki, userPath); - LOG.info("Ephemeral path $c1"); - String c2 = createEphemeralChild(zki, userPath); - LOG.info("Ephemeral path $c2"); - List clusters = zki.getClusters(); - assertEquals(2, clusters.size()); - assertTrue((c1.endsWith(clusters.get(0)) && c2.endsWith(clusters.get(1))) || - (c1.endsWith(clusters.get(1)) && c2.endsWith(clusters.get(0)))); - } - - //@Test - public void testCreateAndDeleteDefaultZKPath() throws Throwable { - MockSliderClient client = new MockSliderClient(); - - String path = client.createZookeeperNodeInner("cl1", true); - zki = client.getLastZKIntegration(); - - String zkPath = ZKIntegration.mkClusterPath(USER, "cl1"); - assertEquals("zkPath must be as expected", zkPath, - "/services/slider/users/" + USER + "/cl1"); - assertEquals(path, zkPath); - assertNull("ZKIntegration should be null.", zki); - zki = createZKIntegrationInstance(getZKBinding(), "cl1", true, false, - CONNECT_TIMEOUT); - assertFalse(zki.exists(zkPath)); - - path = client.createZookeeperNodeInner("cl1", false); - zki = client.getLastZKIntegration(); - assertNotNull(zki); - assertEquals("zkPath must be as expected", zkPath, - "/services/slider/users/" + USER + "/cl1"); - assertEquals(path, zkPath); - assertTrue(zki.exists(zkPath)); - zki.createPath(zkPath, "/cn", ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode - .PERSISTENT); - assertTrue(zki.exists(zkPath + "/cn")); - client.deleteZookeeperNode("cl1"); - assertFalse(zki.exists(zkPath)); - } - - public static String createEphemeralChild(ZKIntegration zki, String userPath) - throws KeeperException, InterruptedException { - return zki.createPath(userPath, "/cluster-", - ZooDefs.Ids.OPEN_ACL_UNSAFE, - CreateMode.EPHEMERAL_SEQUENTIAL); - } - - /** - * Test slider client that overriddes ZK client. - */ - public class MockSliderClient extends SliderClient { - private ZKIntegration zki; - - @Override - public String getUsername() { - return USER; - } - - @Override - protected ZKIntegration getZkClient(String clusterName, String user) { - try { - zki = createZKIntegrationInstance(getZKBinding(), clusterName, true, - false, CONNECT_TIMEOUT); - } catch (Exception e) { - fail("creating ZKIntergration threw an exception"); - } - return zki; - } - - @Override - public Configuration getConfig() { - return new Configuration(); - } - - public ZKIntegration getLastZKIntegration() { - return zki; - } - - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java deleted file mode 100644 index 54c3576..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java +++ /dev/null @@ -1,220 +0,0 @@ -/* - * 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.slider.core.registry.docstore; - -import com.google.common.base.Charsets; -import org.apache.commons.io.FileUtils; -import org.apache.hadoop.fs.Path; -import org.apache.slider.common.tools.SliderFileSystem; -import org.codehaus.jackson.map.ObjectMapper; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.powermock.api.easymock.PowerMock; -import org.yaml.snakeyaml.Yaml; - -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.expect; -import static org.mockito.Matchers.anyString; -import static org.powermock.api.easymock.PowerMock.createNiceMock; - -public class TestPublishedConfigurationOutputter { - private static HashMap config = new HashMap<>(); - - @Rule - public TemporaryFolder tmpDir = new TemporaryFolder(); - - @Before - public void setup() { - config.put("key1", "val1"); - } - - //@Test - public void testJson() throws IOException { - PublishedConfigurationOutputter configurationOutputter = - PublishedConfigurationOutputter.createOutputter(ConfigFormat.JSON, - new PublishedConfiguration("description", - config.entrySet())); - - String output = configurationOutputter.asString().replaceAll("( |\\r|\\n)", - ""); - assert "{\"key1\":\"val1\"}".equals(output); - - File file = tmpDir.newFile(); - configurationOutputter.save(file); - - ObjectMapper mapper = new ObjectMapper(); - @SuppressWarnings("unchecked") - Map read = mapper.readValue(file, Map.class); - assert 1 == read.size(); - assert "val1".equals(read.get("key1")); - } - - //@Test - public void testXml() throws IOException { - PublishedConfigurationOutputter configurationOutputter = - PublishedConfigurationOutputter.createOutputter(ConfigFormat.XML, - new PublishedConfiguration("description", - config.entrySet())); - - String output = configurationOutputter.asString().replaceAll("( |\\r|\\n)", - ""); - assert output.contains("key1val1"); - - File file = tmpDir.newFile(); - configurationOutputter.save(file); - - assert FileUtils.readFileToString(file, Charsets.UTF_8) - .replaceAll("( |\\r|\\n)", "") - .contains("key1val1"); - } - - //@Test - public void testHadoopXml() throws IOException { - PublishedConfigurationOutputter configurationOutputter = - PublishedConfigurationOutputter.createOutputter(ConfigFormat.HADOOP_XML, - new PublishedConfiguration("description", - config.entrySet())); - - String output = configurationOutputter.asString().replaceAll("( |\\r|\\n)", - ""); - assert output.contains("key1val1"); - - File file = tmpDir.newFile(); - configurationOutputter.save(file); - - assert FileUtils.readFileToString(file, Charsets.UTF_8) - .replaceAll("( |\\r|\\n)", "") - .contains("key1val1"); - } - - //@Test - public void testProperties() throws IOException { - PublishedConfigurationOutputter configurationOutputter = - PublishedConfigurationOutputter.createOutputter(ConfigFormat.PROPERTIES, - new PublishedConfiguration("description", - config.entrySet())); - - String output = configurationOutputter.asString(); - assert output.contains("key1=val1"); - - File file = tmpDir.newFile(); - configurationOutputter.save(file); - - Properties properties = new Properties(); - FileInputStream fis = null; - try { - fis = new FileInputStream(file); - properties.load(fis); - } finally { - if (fis != null) { - fis.close(); - } - } - assert 1 == properties.size(); - assert "val1".equals(properties.getProperty("key1")); - } - - //@Test - public void testYaml() throws IOException { - PublishedConfigurationOutputter configurationOutputter = - PublishedConfigurationOutputter.createOutputter(ConfigFormat.YAML, - new PublishedConfiguration("description", - config.entrySet())); - - String output = configurationOutputter.asString().replaceAll("(\\r|\\n)", - ""); - assert "key1: val1".equals(output); - - File file = tmpDir.newFile(); - configurationOutputter.save(file); - - Yaml yaml = new Yaml(); - FileInputStream fis = null; - Map read; - try { - fis = new FileInputStream(file); - read = (Map) yaml.load(fis); - } finally { - if (fis != null) { - fis.close(); - } - } - assert 1 == read.size(); - assert "val1".equals(read.get("key1")); - } - - //@Test - public void testEnv() throws IOException { - HashMap envConfig = new HashMap<>(config); - envConfig.put("content", "content {{key1}} "); - - PublishedConfigurationOutputter configurationOutputter = - PublishedConfigurationOutputter.createOutputter(ConfigFormat.ENV, - new PublishedConfiguration("description", - envConfig.entrySet())); - - String output = configurationOutputter.asString(); - assert "content val1 ".equals(output); - - File file = tmpDir.newFile(); - configurationOutputter.save(file); - - assert "content val1 ".equals(FileUtils.readFileToString(file, - Charsets.UTF_8)); - } - - //@Test - public void testTemplate1() throws IOException { - HashMap templateConfig = new HashMap<>(config); - templateConfig.put(ConfigUtils.TEMPLATE_FILE, "templateFileName"); - - SliderFileSystem fileSystem = createNiceMock(SliderFileSystem.class); - expect(fileSystem.buildResourcePath(anyString())).andReturn(new Path("path")).anyTimes(); - expect(fileSystem.isFile(anyObject(Path.class))).andReturn(true).anyTimes(); - expect(fileSystem.cat(anyObject(Path.class))).andReturn("content {{key1}}\n more ${key1} content").anyTimes(); - - PowerMock.replay(fileSystem); - - ConfigUtils.prepConfigForTemplateOutputter(ConfigFormat.TEMPLATE, - templateConfig, fileSystem, "clusterName", null); - PublishedConfigurationOutputter configurationOutputter = - PublishedConfigurationOutputter.createOutputter(ConfigFormat.TEMPLATE, - new PublishedConfiguration("description", - templateConfig.entrySet())); - - String output = configurationOutputter.asString(); - assert "content val1\n more val1 content".equals(output); - - File file = tmpDir.newFile(); - configurationOutputter.save(file); - - PowerMock.verify(fileSystem); - - assert "content val1\n more val1 content".equals( - FileUtils.readFileToString(file, Charsets.UTF_8)); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/other/TestFilesystemPermissions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/other/TestFilesystemPermissions.java deleted file mode 100644 index 1e5d769..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/other/TestFilesystemPermissions.java +++ /dev/null @@ -1,263 +0,0 @@ -/* - * 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.slider.other; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileContext; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.UnsupportedFileSystemException; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.util.DiskChecker; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService; -import org.apache.slider.utils.YarnMiniClusterTestBase; -import org.junit.After; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * This test class exists to look at permissions of the filesystem, especially - * that created by Mini YARN clusters. On some windows jenkins machines, - * YARN actions were failing as the directories had the wrong permissions - * (i.e. too lax) - */ -public class TestFilesystemPermissions extends YarnMiniClusterTestBase { - - private static final Logger LOG = LoggerFactory.getLogger( - TestFilesystemPermissions.class); - - private List filesToDelete = new ArrayList<>(); - - @After - public void deleteFiles() { - for (File f : filesToDelete) { - FileUtil.fullyDelete(f, true); - } - } - - //@Test - public void testJavaFSOperations() throws Throwable { - assertNativeLibrariesPresent(); - File subdir = testDir(); - subdir.mkdir(); - assertTrue(subdir.isDirectory()); - assertTrue(FileUtil.canRead(subdir)); - assertTrue(FileUtil.canWrite(subdir)); - assertTrue(FileUtil.canExecute(subdir)); - } - - //@Test - public void testDiskCheckerOperations() throws Throwable { - assertNativeLibrariesPresent(); - File subdir = testDir(); - subdir.mkdir(); - DiskChecker checker = new DiskChecker(); - checker.checkDir(subdir); - } - - //@Test - public void testDiskCheckerMkdir() throws Throwable { - assertNativeLibrariesPresent(); - File subdir = testDir(); - subdir.mkdirs(); - DiskChecker checker = new DiskChecker(); - checker.checkDir(subdir); - } - - /** - * Get a test dir for this method; one that will be deleted on teardown. - * @return a filename unique to this test method - */ - File testDir() { - File parent = new File("target/testfspermissions"); - parent.mkdir(); - File testdir = new File(parent, methodName.getMethodName()); - filesToDelete.add(testdir); - return testdir; - } - - - //@Test - public void testPermsMap() throws Throwable { - File dir = testDir(); - String diruri = dir.toURI().toString(); - FileContext lfs = createLocalFS(dir, getConfiguration()); - getLocalDirsPathPermissionsMap(lfs, diruri); - } - - //@Test - public void testInitLocaldir() throws Throwable { - File dir = testDir(); - String diruri = dir.toURI().toString(); - FileContext lfs = createLocalFS(dir, getConfiguration()); - initializeLocalDir(lfs, diruri); - List localDirs = getInitializedLocalDirs(lfs, Arrays.asList( - diruri)); - assertEquals(1, localDirs.size()); - } - - - //@Test - public void testValidateMiniclusterPerms() throws Throwable { - int numLocal = 1; - String cluster = createMiniCluster("", getConfiguration(), 1, numLocal, 1, - false); - File workDir = getMiniCluster().getTestWorkDir(); - List localdirs = new ArrayList<>(); - for (File file : workDir.listFiles()) { - if (file.isDirectory() && file.getAbsolutePath().contains("-local")) { - // local dir - localdirs.add(file); - } - } - assertEquals(numLocal, localdirs.size()); - FileContext lfs = createLocalFS(workDir, getConfiguration()); - for (File file : localdirs) { - checkLocalDir(lfs, file.toURI().toString()); - } - } - - FileContext createLocalFS(File dir, Configuration conf) - throws UnsupportedFileSystemException { - return FileContext.getFileContext(dir.toURI(), conf); - } - - /** - * Extracted from ResourceLocalizationService. - * @param lfs - * @param localDir - * @return perms map - * @see ResourceLocalizationService - */ - private Map getLocalDirsPathPermissionsMap( - FileContext lfs, - String localDir) { - Map localDirPathFsPermissionsMap = new HashMap<>(); - - FsPermission defaultPermission = - FsPermission.getDirDefault().applyUMask(lfs.getUMask()); - FsPermission nmPrivatePermission = - ResourceLocalizationService.NM_PRIVATE_PERM.applyUMask(lfs.getUMask()); - - Path userDir = new Path(localDir, ContainerLocalizer.USERCACHE); - Path fileDir = new Path(localDir, ContainerLocalizer.FILECACHE); - Path sysDir = new Path( - localDir, - ResourceLocalizationService.NM_PRIVATE_DIR); - - localDirPathFsPermissionsMap.put(userDir, defaultPermission); - localDirPathFsPermissionsMap.put(fileDir, defaultPermission); - localDirPathFsPermissionsMap.put(sysDir, nmPrivatePermission); - return localDirPathFsPermissionsMap; - } - - private boolean checkLocalDir(FileContext lfs, String localDir) - throws IOException { - - Map pathPermissionMap = - getLocalDirsPathPermissionsMap(lfs, localDir); - - for (Map.Entry entry : pathPermissionMap.entrySet()) { - FileStatus status; - status = lfs.getFileStatus(entry.getKey()); - - if (!status.getPermission().equals(entry.getValue())) { - String msg = - "Permissions incorrectly set for dir " + entry.getKey() + - ", should be " + entry.getValue() + ", actual value = " + - status.getPermission(); - throw new YarnRuntimeException(msg); - } - } - return true; - } - - - private void initializeLocalDir(FileContext lfs, String localDir) - throws IOException { - - Map pathPermissionMap = - getLocalDirsPathPermissionsMap(lfs, localDir); - for (Map.Entry entry : pathPermissionMap.entrySet()) { - FileStatus status; - try { - status = lfs.getFileStatus(entry.getKey()); - } catch (FileNotFoundException fs) { - status = null; - } - - if (status == null) { - lfs.mkdir(entry.getKey(), entry.getValue(), true); - status = lfs.getFileStatus(entry.getKey()); - } - FsPermission perms = status.getPermission(); - if (!perms.equals(entry.getValue())) { - lfs.setPermission(entry.getKey(), entry.getValue()); - } - } - } - - synchronized private List getInitializedLocalDirs(FileContext lfs, - List dirs) throws IOException { - List checkFailedDirs = new ArrayList(); - for (String dir : dirs) { - try { - checkLocalDir(lfs, dir); - } catch (YarnRuntimeException e) { - checkFailedDirs.add(dir); - } - } - for (String dir : checkFailedDirs) { - LOG.info("Attempting to initialize " + dir); - initializeLocalDir(lfs, dir); - checkLocalDir(lfs, dir); - } - return dirs; - } - - - private void createDir(FileContext localFs, Path dir, FsPermission perm) - throws IOException { - if (dir == null) { - return; - } - try { - localFs.getFileStatus(dir); - } catch (FileNotFoundException e) { - createDir(localFs, dir.getParent(), perm); - localFs.mkdir(dir, perm, false); - if (!perm.equals(perm.applyUMask(localFs.getUMask()))) { - localFs.setPermission(dir, perm); - } - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/other/TestLocalDirStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/other/TestLocalDirStatus.java deleted file mode 100644 index a45b27d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/other/TestLocalDirStatus.java +++ /dev/null @@ -1,166 +0,0 @@ -/* - * 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.slider.other; - -import org.apache.slider.utils.SliderTestUtils; -import org.apache.slider.utils.TestUtility; -import org.junit.Test; - -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; - -/** - * This test exists to diagnose local FS permissions. - */ -public class TestLocalDirStatus extends SliderTestUtils { - - - public static final int SIZE = 0x200000; - - //@Test - public void testTempDir() throws Throwable { - File tmpf = null; - try { - tmpf = File.createTempFile("testl", ".bin"); - createAndReadFile(tmpf, SIZE); - tmpf.delete(); - assertFalse(tmpf.exists()); - } finally { - if (tmpf != null) { - tmpf.delete(); - } - } - } - - //@Test - public void testTargetDir() throws Throwable { - File target = target(); - File tmpf = null; - try { - tmpf = File.createTempFile("testl", ".bin", target); - createAndReadFile(tmpf, SIZE); - tmpf.delete(); - assertFalse(tmpf.exists()); - } finally { - if (tmpf != null) { - tmpf.delete(); - } - - } - } - - public File target() { - File target = new File("target").getAbsoluteFile(); - assertTrue(target.exists()); - return target; - } - - //@Test - public void testRenameInTargetDir() throws Throwable { - File target = target(); - File tmpf = null; - File dst= null; - try { - tmpf = File.createTempFile("testl", ".bin", target); - dst = File.createTempFile("test-dest", ".bin", target); - createRenameAndReadFile(tmpf, dst, SIZE); - assertFalse(tmpf.exists()); - dst.delete(); - } finally { - if (tmpf != null) { - tmpf.delete(); - } - if (dst != null) { - dst.delete(); - } - } - } - - //@Test - public void testRenameInTmpDir() throws Throwable { - File tmpf = null; - File dst= null; - try { - tmpf = File.createTempFile("testl", ".bin"); - dst = File.createTempFile("test-dest", ".bin"); - createRenameAndReadFile(tmpf, dst, SIZE); - assertFalse(tmpf.exists()); - dst.delete(); - } finally { - if (tmpf != null) { - tmpf.delete(); - } - if (dst != null) { - dst.delete(); - } - } - } - - protected void createAndReadFile(File path, int len) throws IOException { - byte[] dataset = TestUtility.dataset(len, 32, 128); - writeFile(path, dataset); - assertTrue(path.exists()); - assertEquals(len, path.length()); - byte[] persisted = readFile(path); - TestUtility.compareByteArrays(dataset, persisted, len); - } - - protected void createRenameAndReadFile(File src, File dst, int len) - throws IOException { - byte[] dataset = TestUtility.dataset(len, 32, 128); - writeFile(src, dataset); - assertTrue(src.exists()); - assertEquals(len, src.length()); - dst.delete(); - assertFalse(dst.exists()); - assertTrue(src.renameTo(dst)); - assertEquals(len, dst.length()); - byte[] persisted = readFile(dst); - TestUtility.compareByteArrays(dataset, persisted, len); - } - - protected void writeFile(File path, byte[] dataset) - throws IOException { - FileOutputStream out = new FileOutputStream(path); - try { - out.write(dataset); - out.flush(); - } finally { - out.close(); - } - } - - protected byte[] readFile(File path) throws IOException { - assertTrue(path.getAbsoluteFile().exists()); - assertTrue(path.getAbsoluteFile().isFile()); - int len = (int)path.length(); - byte[] dataset = new byte[len]; - FileInputStream ins = new FileInputStream(path); - try { - ins.read(dataset); - } finally { - ins.close(); - } - return dataset; - } - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/TestAbstractClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/TestAbstractClientProvider.java deleted file mode 100644 index 7bb8707..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/TestAbstractClientProvider.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.slider.providers; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.yarn.service.provider.AbstractClientProvider; -import org.apache.slider.api.resource.Artifact; -import org.apache.slider.api.resource.ConfigFile; -import org.apache.slider.api.resource.ConfigFile.TypeEnum; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.createNiceMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; - -/** - * Test the AbstractClientProvider shared methods. - */ -public class TestAbstractClientProvider { - private static final String EXCEPTION_PREFIX = "Should have thrown " + - "exception: "; - private static final String NO_EXCEPTION_PREFIX = "Should not have thrown " + - "exception: "; - - private static class ClientProvider extends AbstractClientProvider { - @Override - public void validateArtifact(Artifact artifact, FileSystem fileSystem) - throws IOException { - } - - @Override - protected void validateConfigFile(ConfigFile configFile, - FileSystem fileSystem) throws IOException { - } - } - - //@Test - public void testConfigFiles() throws IOException { - ClientProvider clientProvider = new ClientProvider(); - FileSystem mockFs = createNiceMock(FileSystem.class); - expect(mockFs.exists(anyObject(Path.class))).andReturn(true).anyTimes(); - replay(mockFs); - - ConfigFile configFile = new ConfigFile(); - List configFiles = new ArrayList<>(); - configFiles.add(configFile); - - try { - clientProvider.validateConfigFiles(configFiles, mockFs); - Assert.fail(EXCEPTION_PREFIX + "null file type"); - } catch (IllegalArgumentException e) { - } - - configFile.setType(TypeEnum.TEMPLATE); - try { - clientProvider.validateConfigFiles(configFiles, mockFs); - Assert.fail(EXCEPTION_PREFIX + "empty src_file for type template"); - } catch (IllegalArgumentException e) { - } - - configFile.setSrcFile("srcfile"); - try { - clientProvider.validateConfigFiles(configFiles, mockFs); - Assert.fail(EXCEPTION_PREFIX + "empty dest file"); - } catch (IllegalArgumentException e) { - } - - configFile.setDestFile("destfile"); - try { - clientProvider.validateConfigFiles(configFiles, mockFs); - } catch (IllegalArgumentException e) { - Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage()); - } - - configFile = new ConfigFile(); - configFile.setType(TypeEnum.JSON); - configFile.setSrcFile(null); - configFile.setDestFile("path/destfile2"); - configFiles.add(configFile); - try { - clientProvider.validateConfigFiles(configFiles, mockFs); - Assert.fail(EXCEPTION_PREFIX + "dest file with multiple path elements"); - } catch (IllegalArgumentException e) { - } - - configFile.setDestFile("/path/destfile2"); - try { - clientProvider.validateConfigFiles(configFiles, mockFs); - } catch (IllegalArgumentException e) { - Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage()); - } - - configFile.setDestFile("destfile"); - try { - clientProvider.validateConfigFiles(configFiles, mockFs); - Assert.fail(EXCEPTION_PREFIX + "duplicate dest file"); - } catch (IllegalArgumentException e) { - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/TestProviderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/TestProviderFactory.java deleted file mode 100644 index c51eee2..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/TestProviderFactory.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.slider.providers; - -import org.apache.hadoop.yarn.service.provider.defaultImpl.DefaultClientProvider; -import org.apache.hadoop.yarn.service.provider.defaultImpl.DefaultProviderFactory; -import org.apache.hadoop.yarn.service.provider.defaultImpl.DefaultProviderService; -import org.apache.hadoop.yarn.service.provider.ProviderFactory; -import org.apache.slider.api.resource.Artifact; -import org.apache.slider.api.resource.Artifact.TypeEnum; -import org.apache.hadoop.yarn.service.provider.docker.DockerClientProvider; -import org.apache.hadoop.yarn.service.provider.docker.DockerProviderFactory; -import org.apache.hadoop.yarn.service.provider.docker.DockerProviderService; -import org.apache.hadoop.yarn.service.provider.tarball.TarballClientProvider; -import org.apache.hadoop.yarn.service.provider.tarball.TarballProviderFactory; -import org.apache.hadoop.yarn.service.provider.tarball.TarballProviderService; -import org.junit.Test; - -import static org.junit.Assert.assertTrue; - -/** - * Test provider factories. - */ -public class TestProviderFactory { - //@Test - public void testDockerFactory() throws Throwable { - ProviderFactory factory = ProviderFactory - .createSliderProviderFactory(new Artifact().type(TypeEnum.DOCKER)); - assertTrue(factory instanceof DockerProviderFactory); - assertTrue(factory.createClientProvider() instanceof DockerClientProvider); - assertTrue(factory.createServerProvider() instanceof DockerProviderService); - assertTrue(ProviderFactory.getProviderService(new Artifact() - .type(TypeEnum.DOCKER)) instanceof DockerProviderService); - } - - //@Test - public void testTarballFactory() throws Throwable { - ProviderFactory factory = ProviderFactory - .createSliderProviderFactory(new Artifact().type(TypeEnum.TARBALL)); - assertTrue(factory instanceof TarballProviderFactory); - assertTrue(factory.createClientProvider() instanceof TarballClientProvider); - assertTrue(factory.createServerProvider() instanceof - TarballProviderService); - assertTrue(ProviderFactory.getProviderService(new Artifact() - .type(TypeEnum.TARBALL)) instanceof TarballProviderService); - } - - //@Test - public void testDefaultFactory() throws Throwable { - ProviderFactory factory = ProviderFactory - .createSliderProviderFactory(null); - assertTrue(factory instanceof DefaultProviderFactory); - assertTrue(factory.createClientProvider() instanceof DefaultClientProvider); - assertTrue(factory.createServerProvider() instanceof DefaultProviderService); - assertTrue(ProviderFactory.getProviderService(null) instanceof - DefaultProviderService); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/registry/TestConfigSetNaming.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/registry/TestConfigSetNaming.java deleted file mode 100644 index 5743119..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/registry/TestConfigSetNaming.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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.slider.registry; - -import org.apache.slider.core.registry.docstore.PublishedConfigSet; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; - -/** - * Test config set name validation. - */ -public class TestConfigSetNaming { - - void assertValid(String name) { - PublishedConfigSet.validateName(name); - } - - void assertInvalid(String name) { - try { - PublishedConfigSet.validateName(name); - Assert.fail("Invalid name was unexpectedly parsed: " + name); - } catch (IllegalArgumentException expected) { - // expected - } - } - - //@Test - public void testLowerCase() throws Throwable { - assertValid("abcdefghijklmnopqrstuvwxyz"); - } - - //@Test - public void testUpperCaseInvalid() throws Throwable { - assertInvalid("ABCDEFGHIJKLMNOPQRSTUVWXYZ"); - } - - //@Test - public void testNumbers() throws Throwable { - assertValid("01234567890"); - } - - //@Test - public void testChars() throws Throwable { - assertValid("a-_+"); - } - - //@Test - public void testInvalids() throws Throwable { - for (String s : Arrays.asList( - "", - " ", - "*", - "a/b", - "b\\a", - "\"", - "'", - "\u0000", - "\u0f00", - "key.value", - "-", - "+", - "_", - "?")) { - assertInvalid(s); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/registry/TestRegistryPaths.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/registry/TestRegistryPaths.java deleted file mode 100644 index 5e6b650..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/registry/TestRegistryPaths.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.slider.registry; - -import org.apache.hadoop.registry.client.binding.RegistryUtils; -import org.apache.slider.core.registry.SliderRegistryUtils; -import org.apache.slider.utils.SliderTestUtils; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -/** - * Test registry paths. - */ -public class TestRegistryPaths { - - //@Test - public void testHomedirKerberos() throws Throwable { - String home = RegistryUtils.homePathForUser("hbase@HADOOP.APACHE.ORG"); - try { - assertEquals("/users/hbase", home); - } catch (AssertionError e) { - SliderTestUtils.skip("homedir filtering not yet in hadoop registry " + - "module"); - } - } - - //@Test - public void testHomedirKerberosHost() throws Throwable { - String home = RegistryUtils.homePathForUser("hbase/localhost@HADOOP" + - ".APACHE.ORG"); - try { - assertEquals("/users/hbase", home); - } catch (AssertionError e) { - SliderTestUtils.skip("homedir filtering not yet in hadoop registry " + - "module"); - } - } - - //@Test - public void testRegistryPathForInstance() throws Throwable { - String path = SliderRegistryUtils.registryPathForInstance("instance"); - assertTrue(path.endsWith("/instance")); - } - - //@Test - public void testPathResolution() throws Throwable { - String home = RegistryUtils.homePathForCurrentUser(); - assertEquals(home, SliderRegistryUtils.resolvePath("~")); - assertEquals(home +"/", SliderRegistryUtils.resolvePath("~/")); - assertEquals(home +"/something", SliderRegistryUtils.resolvePath( - "~/something")); - assertEquals("~unresolved", SliderRegistryUtils.resolvePath( - "~unresolved")); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/actions/TestActions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/actions/TestActions.java deleted file mode 100644 index a63f4b6..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/actions/TestActions.java +++ /dev/null @@ -1,246 +0,0 @@ -/* - * 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.slider.server.appmaster.actions; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.ServiceOperations; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.services.workflow.ServiceThreadFactory; -import org.apache.slider.server.services.workflow.WorkflowExecutorService; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -/** - * Test AM actions. - */ -public class TestActions { - protected static final Logger LOG = - LoggerFactory.getLogger(TestActions.class); - - private QueueService queues; - private WorkflowExecutorService executorService; - - - @Before - public void createService() { - queues = new QueueService(); - - Configuration conf = new Configuration(); - queues.init(conf); - - queues.start(); - - executorService = new WorkflowExecutorService<>("AmExecutor", - Executors.newCachedThreadPool( - new ServiceThreadFactory("AmExecutor", true))); - - executorService.init(conf); - executorService.start(); - } - - @After - public void destroyService() { - ServiceOperations.stop(executorService); - ServiceOperations.stop(queues); - } - - //@Test - public void testBasicService() throws Throwable { - queues.start(); - } - - //@Test - public void testDelayLogic() throws Throwable { - ActionNoteExecuted action = new ActionNoteExecuted("", 1000); - long now = System.currentTimeMillis(); - - long delay = action.getDelay(TimeUnit.MILLISECONDS); - assertTrue(delay >= 800); - assertTrue(delay <= 1800); - - ActionNoteExecuted a2 = new ActionNoteExecuted("a2", 10000); - assertTrue(action.compareTo(a2) < 0); - assertTrue(a2.compareTo(action) > 0); - assertEquals(0, action.compareTo(action)); - - } - - //@Test - public void testActionDelayedExecutorTermination() throws Throwable { - long start = System.currentTimeMillis(); - - ActionStopQueue stopAction = new ActionStopQueue(1000); - queues.scheduledActions.add(stopAction); - queues.run(); - AsyncAction take = queues.actionQueue.take(); - assertEquals(take, stopAction); - long stop = System.currentTimeMillis(); - assertTrue(stop - start > 500); - assertTrue(stop - start < 1500); - } - - //@Test - public void testImmediateQueue() throws Throwable { - ActionNoteExecuted noteExecuted = new ActionNoteExecuted("executed", 0); - queues.put(noteExecuted); - queues.put(new ActionStopQueue(0)); - QueueExecutor ex = new QueueExecutor(queues); - ex.run(); - assertTrue(queues.actionQueue.isEmpty()); - assertTrue(noteExecuted.executed.get()); - } - - //@Test - public void testActionOrdering() throws Throwable { - - ActionNoteExecuted note1 = new ActionNoteExecuted("note1", 500); - ActionStopQueue stop = new ActionStopQueue(1500); - ActionNoteExecuted note2 = new ActionNoteExecuted("note2", 800); - - List actions = Arrays.asList(note1, stop, note2); - Collections.sort(actions); - assertEquals(actions.get(0), note1); - assertEquals(actions.get(1), note2); - assertEquals(actions.get(2), stop); - } - - //@Test - public void testDelayedQueueWithReschedule() throws Throwable { - - ActionNoteExecuted note1 = new ActionNoteExecuted("note1", 500); - ActionStopQueue stop = new ActionStopQueue(1500); - ActionNoteExecuted note2 = new ActionNoteExecuted("note2", 800); - - assertTrue(note2.compareTo(stop) < 0); - assertTrue(note1.getNanos() < note2.getNanos()); - assertTrue(note2.getNanos() < stop.getNanos()); - queues.schedule(note1); - queues.schedule(note2); - queues.schedule(stop); - // async to sync expected to run in order - runQueuesToCompletion(); - assertTrue(note1.executed.get()); - assertTrue(note2.executed.get()); - } - - public void runQueuesToCompletion() { - queues.run(); - assertTrue(queues.scheduledActions.isEmpty()); - assertFalse(queues.actionQueue.isEmpty()); - QueueExecutor ex = new QueueExecutor(queues); - ex.run(); - // flush all stop commands from the queue - queues.flushActionQueue(ActionStopQueue.class); - - assertTrue(queues.actionQueue.isEmpty()); - } - - //@Test - public void testRenewedActionFiresOnceAtLeast() throws Throwable { - ActionNoteExecuted note1 = new ActionNoteExecuted("note1", 500); - RenewingAction renewer = new RenewingAction( - note1, - 500, - 100, - TimeUnit.MILLISECONDS, - 3); - queues.schedule(renewer); - ActionStopQueue stop = new ActionStopQueue(4, TimeUnit.SECONDS); - queues.schedule(stop); - // this runs all the delayed actions FIRST, so can't be used - // to play tricks of renewing actions ahead of the stop action - runQueuesToCompletion(); - assertEquals(1, renewer.executionCount.intValue()); - assertEquals(1, note1.executionCount.intValue()); - // assert the renewed item is back in - assertTrue(queues.scheduledActions.contains(renewer)); - } - - - //@Test - public void testRenewingActionOperations() throws Throwable { - ActionNoteExecuted note1 = new ActionNoteExecuted("note1", 500); - RenewingAction renewer = new RenewingAction( - note1, - 100, - 100, - TimeUnit.MILLISECONDS, - 3); - queues.renewing("note", renewer); - assertTrue(queues.removeRenewingAction("note")); - queues.stop(); - assertTrue(queues.waitForServiceToStop(10000)); - } - - /** - * Test action. - */ - public class ActionNoteExecuted extends AsyncAction { - private final AtomicBoolean executed = new AtomicBoolean(false); - private final AtomicLong executionTimeNanos = new AtomicLong(); - private final AtomicLong executionCount = new AtomicLong(); - - public ActionNoteExecuted(String text, int delay) { - super(text, delay); - } - - @Override - public void execute( - SliderAppMaster appMaster, - QueueAccess queueService, - AppState appState) throws Exception { - LOG.info("Executing {}", name); - executed.set(true); - executionTimeNanos.set(System.nanoTime()); - executionCount.incrementAndGet(); - LOG.info(this.toString()); - - synchronized (this) { - this.notify(); - } - } - - @Override - public String toString() { - return super.toString() + " executed=" + executed.get() + "; count=" + - executionCount.get() + ";"; - } - - public long getExecutionCount() { - return executionCount.get(); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/BaseMockAppStateAATest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/BaseMockAppStateAATest.java deleted file mode 100644 index 6f4ca42..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/BaseMockAppStateAATest.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.api.resource.Application; -import org.apache.slider.providers.PlacementPolicy; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.state.RoleStatus; - -import static org.apache.slider.api.ResourceKeys.COMPONENT_PLACEMENT_POLICY; - -/** - * Class for basis of Anti-affine placement tests; sets up role2 - * for anti-affinity. - */ -public class BaseMockAppStateAATest extends BaseMockAppStateTest - implements MockRoles { - - /** Role status for the base AA role. */ - private RoleStatus aaRole; - - /** Role status for the AA role requiring a node with the gpu label. */ - private RoleStatus gpuRole; - - @Override - public Application buildApplication() { - Application application = factory.newApplication(0, 0, 0) - .name(getValidTestName()); - application.getComponent(ROLE1).getConfiguration().setProperty( - COMPONENT_PLACEMENT_POLICY, Integer.toString(PlacementPolicy - .ANTI_AFFINITY_REQUIRED)); - application.getComponent(ROLE1).getConfiguration().setProperty( - ResourceKeys.YARN_LABEL_EXPRESSION, LABEL_GPU); - application.getComponent(ROLE2).getConfiguration().setProperty( - COMPONENT_PLACEMENT_POLICY, Integer.toString(PlacementPolicy - .ANTI_AFFINITY_REQUIRED)); - return application; - } - - - @Override - public void setup() throws Exception { - super.setup(); - aaRole = lookupRole(ROLE2); - gpuRole = lookupRole(ROLE1); - } - - protected RoleStatus getAaRole() { - return aaRole; - } - - protected RoleStatus getGpuRole() { - return gpuRole; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAAOvercapacity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAAOvercapacity.java deleted file mode 100644 index e0f7c1f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAAOvercapacity.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.slider.core.main.LauncherExitCodes; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.model.mock.MockYarnEngine; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.NodeMap; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; - -/** - * Test Anti-affine placement with a cluster of size 1. - */ -public class TestMockAppStateAAOvercapacity extends BaseMockAppStateAATest - implements MockRoles { - - private static final int NODES = 1; - - @Override - public MockYarnEngine createYarnEngine() { - return new MockYarnEngine(NODES, 1); - } - - void assertAllContainersAA() { - assertAllContainersAA(getAaRole().getKey()); - } - - /** - * - * @throws Throwable - */ - //@Test - public void testOvercapacityRecovery() throws Throwable { - RoleStatus aaRole = getAaRole(); - - describe("Ask for 1 more than the no of available nodes;" + - "verify the state. kill the allocated container and review"); - //more than expected - int desired = 3; - aaRole.setDesired(desired); - assertTrue(appState.getRoleHistory().canPlaceAANodes()); - - //first request - List operations = - appState.reviewRequestAndReleaseNodes(); - assertTrue(aaRole.isAARequestOutstanding()); - assertEquals(1, aaRole.getRequested()); - assertEquals(desired - 1, aaRole.getAAPending()); - List operationsOut = new ArrayList<>(); - // allocate and re-submit - List instances = submitOperations(operations, - EMPTY_ID_LIST, operationsOut); - assertEquals(1, instances.size()); - assertAllContainersAA(); - - // expect an outstanding AA request to be unsatisfied - assertTrue(aaRole.getRunning() < aaRole.getDesired()); - assertEquals(0, aaRole.getRequested()); - assertFalse(aaRole.isAARequestOutstanding()); - assertEquals(desired - 1, aaRole.getAAPending()); - List allocatedContainers = engine.execute(operations, - EMPTY_ID_LIST); - assertEquals(0, allocatedContainers.size()); - - // now lets trigger a failure - NodeMap nodemap = cloneNodemap(); - assertEquals(1, nodemap.size()); - - RoleInstance instance = instances.get(0); - ContainerId cid = instance.getContainerId(); - - AppState.NodeCompletionResult result = appState.onCompletedContainer( - containerStatus(cid, LauncherExitCodes.EXIT_TASK_LAUNCH_FAILURE)); - assertTrue(result.containerFailed); - - assertEquals(1, aaRole.getFailed()); - assertEquals(0, aaRole.getRunning()); - List availablePlacements = appState.getRoleHistory() - .findNodeForNewAAInstance(aaRole); - assertEquals(1, availablePlacements.size()); - describe("expecting a successful review with available placements of " + - availablePlacements); - operations = appState.reviewRequestAndReleaseNodes(); - assertEquals(1, operations.size()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAAPlacement.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAAPlacement.java deleted file mode 100644 index 395ff22..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateAAPlacement.java +++ /dev/null @@ -1,380 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.NodeState; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; -import org.apache.slider.api.types.NodeInformation; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.providers.PlacementPolicy; -import org.apache.slider.server.appmaster.model.mock.MockAppState; -import org.apache.slider.server.appmaster.model.mock.MockFactory; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.model.mock.MockYarnEngine; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.AppState.NodeUpdatedOutcome; -import org.apache.slider.server.appmaster.state.AppStateBindingInfo; -import org.apache.slider.server.appmaster.state.ContainerAssignment; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; - -import static org.apache.slider.api.ResourceKeys.COMPONENT_PLACEMENT_POLICY; -import static org.apache.slider.server.appmaster.model.mock.MockFactory.AAROLE_2; - -/** - * Test Anti-affine placement. - */ -public class TestMockAppStateAAPlacement extends BaseMockAppStateAATest - implements MockRoles { - private static final Logger LOG = - LoggerFactory.getLogger(TestMockAppStateAAPlacement.class); - - private static final int NODES = 3; - - /** - * The YARN engine has a cluster with very few nodes (3) and lots of - * containers, so if AA placement isn't working, there will be affine - * placements surfacing. - * @return - */ - @Override - public MockYarnEngine createYarnEngine() { - return new MockYarnEngine(NODES, 8); - } - - /** - * This is the simplest AA allocation: no labels, so allocate anywhere. - * @throws Throwable - */ - //@Test - public void testAllocateAANoLabel() throws Throwable { - RoleStatus aaRole = getAaRole(); - - assertTrue(cloneNodemap().size() > 0); - - // want multiple instances, so there will be iterations - aaRole.setDesired(2); - - List ops = appState.reviewRequestAndReleaseNodes(); - AMRMClient.ContainerRequest request = getSingleRequest(ops); - assertFalse(request.getRelaxLocality()); - assertEquals(request.getNodes().size(), engine.getCluster() - .getClusterSize()); - assertNull(request.getRacks()); - assertNotNull(request.getCapability()); - - Container allocated = engine.allocateContainer(request); - - // notify the container ane expect - List assignments = new ArrayList<>(); - List operations = new ArrayList<>(); - appState.onContainersAllocated(Arrays.asList(allocated), assignments, - operations); - - String host = allocated.getNodeId().getHost(); - NodeInstance hostInstance = cloneNodemap().get(host); - assertEquals(1, hostInstance.get(aaRole.getKey()).getStarting()); - assertFalse(hostInstance.canHost(aaRole.getKey(), "")); - assertFalse(hostInstance.canHost(aaRole.getKey(), null)); - - // assignment - assertEquals(1, assignments.size()); - - // verify the release matches the allocation - assertEquals(2, operations.size()); - assertNotNull(getCancel(operations, 0).getCapability().equals(allocated - .getResource())); - - // we also expect a new allocation request to have been issued - - ContainerRequest req2 = getRequest(operations, 1); - assertEquals(req2.getNodes().size(), engine.getCluster() - .getClusterSize() - 1); - - assertFalse(req2.getNodes().contains(host)); - assertFalse(request.getRelaxLocality()); - - // verify the pending couner is down - assertEquals(0L, aaRole.getAAPending()); - Container allocated2 = engine.allocateContainer(req2); - - // placement must be on a different host - assertNotEquals(allocated2.getNodeId(), allocated.getNodeId()); - - ContainerAssignment assigned = assignments.get(0); - Container container = assigned.container; - RoleInstance ri = roleInstance(assigned); - //tell the app it arrived - appState.containerStartSubmitted(container, ri); - assertNotNull(appState.onNodeManagerContainerStarted(container.getId())); - ops = appState.reviewRequestAndReleaseNodes(); - assertEquals(0, ops.size()); - assertAllContainersAA(); - - // identify those hosts with an aa role on - Map naming = appState.buildNamingMap(); - assertEquals(3, naming.size()); - - String name = aaRole.getName(); - assertEquals(name, naming.get(aaRole.getKey())); - Map info = - appState.getRoleHistory().getNodeInformationSnapshot(naming); - assertTrue(SliderUtils.isNotEmpty(info)); - - NodeInformation nodeInformation = info.get(host); - assertNotNull(nodeInformation); - assertTrue(SliderUtils.isNotEmpty(nodeInformation.entries)); - assertNotNull(nodeInformation.entries.get(name)); - assertEquals(1, nodeInformation.entries.get(name).live); - } - - //@Test - public void testAllocateFlexUp() throws Throwable { - RoleStatus aaRole = getAaRole(); - - // want multiple instances, so there will be iterations - aaRole.setDesired(2); - List ops = appState.reviewRequestAndReleaseNodes(); - getSingleRequest(ops); - assertEquals(1, aaRole.getRequested()); - assertEquals(1, aaRole.getAAPending()); - assertEquals(aaRole.getActualAndRequested() + aaRole - .getAAPending(), aaRole.getDesired()); - - // now trigger that flex up - aaRole.setDesired(3); - - // expect: no new reqests, pending count ++ - List ops2 = appState.reviewRequestAndReleaseNodes(); - assertTrue(ops2.isEmpty()); - assertEquals(aaRole.getRunning() + aaRole.getAAPending() + - aaRole.getOutstandingAARequestCount(), aaRole.getDesired()); - - // 1 outstanding - assertEquals(0, aaRole.getRunning()); - assertTrue(aaRole.isAARequestOutstanding()); - // and one AA - assertEquals(2, aaRole.getAAPending()); - assertAllContainersAA(); - - // next iter - assertEquals(1, submitOperations(ops, EMPTY_ID_LIST, ops2).size()); - assertEquals(2, ops2.size()); - assertEquals(1, aaRole.getAAPending()); - assertAllContainersAA(); - - assertEquals(0, appState.reviewRequestAndReleaseNodes().size()); - // now trigger the next execution cycle - List ops3 = new ArrayList<>(); - assertEquals(1, submitOperations(ops2, EMPTY_ID_LIST, ops3).size()); - assertEquals(2, ops3.size()); - assertEquals(0, aaRole.getAAPending()); - assertAllContainersAA(); - - } - - //@Test - public void testAllocateFlexDownDecrementsPending() throws Throwable { - RoleStatus aaRole = getAaRole(); - - // want multiple instances, so there will be iterations - aaRole.setDesired(2); - List ops = appState.reviewRequestAndReleaseNodes(); - getSingleRequest(ops); - assertEquals(1, aaRole.getAAPending()); - assertTrue(aaRole.isAARequestOutstanding()); - - // flex down so that the next request should be cancelled - aaRole.setDesired(1); - - // expect: no new requests, pending count -- - List ops2 = appState.reviewRequestAndReleaseNodes(); - assertTrue(ops2.isEmpty()); - assertTrue(aaRole.isAARequestOutstanding()); - assertEquals(0, aaRole.getAAPending()); - assertAllContainersAA(); - - // next iter - submitOperations(ops, EMPTY_ID_LIST, ops2).size(); - assertEquals(1, ops2.size()); - assertAllContainersAA(); - } - - /** - * Here flex down while there is only one outstanding request. - * The outstanding flex should be cancelled - * @throws Throwable - */ - //@Test - public void testAllocateFlexDownForcesCancel() throws Throwable { - RoleStatus aaRole = getAaRole(); - - // want multiple instances, so there will be iterations - aaRole.setDesired(1); - List ops = appState.reviewRequestAndReleaseNodes(); - getSingleRequest(ops); - assertEquals(1, aaRole.getRequested()); - assertEquals(0, aaRole.getAAPending()); - assertTrue(aaRole.isAARequestOutstanding()); - - // flex down so that the next request should be cancelled - aaRole.setDesired(0); - // expect: no new requests, pending count -- - List ops2 = appState.reviewRequestAndReleaseNodes(); - assertEquals(0, aaRole.getRequested()); - assertEquals(0, aaRole.getAAPending()); - assertFalse(aaRole.isAARequestOutstanding()); - assertEquals(1, ops2.size()); - getSingleCancel(ops2); - - // next iter - submitOperations(ops, EMPTY_ID_LIST, ops2).size(); - getSingleRelease(ops2); - } - - void assertAllContainersAA() { - assertAllContainersAA(getAaRole().getKey()); - } - - /** - * - * @throws Throwable - */ - //@Test - public void testAskForTooMany() throws Throwable { - RoleStatus aaRole = getAaRole(); - - describe("Ask for 1 more than the no of available nodes;" + - " expect the final request to be unsatisfied until the cluster " + - "changes size"); - //more than expected - aaRole.setDesired(NODES + 1); - List operations = appState - .reviewRequestAndReleaseNodes(); - assertTrue(aaRole.isAARequestOutstanding()); - assertEquals(NODES, aaRole.getAAPending()); - for (int i = 0; i < NODES; i++) { - String iter = "Iteration " + i + " role = " + aaRole; - LOG.info(iter); - List operationsOut = new ArrayList<>(); - assertEquals(1, submitOperations(operations, EMPTY_ID_LIST, - operationsOut).size()); - operations = operationsOut; - if (i + 1 < NODES) { - assertEquals(2, operations.size()); - } else { - assertEquals(1, operations.size()); - } - assertAllContainersAA(); - } - // expect an outstanding AA request to be unsatisfied - assertTrue(aaRole.getRunning() < aaRole.getDesired()); - assertEquals(0, aaRole.getRequested()); - assertFalse(aaRole.isAARequestOutstanding()); - List allocatedContainers = engine.execute(operations, - EMPTY_ID_LIST); - assertEquals(0, allocatedContainers.size()); - // in a review now, no more requests can be generated, as there is no - // space for AA placements, even though there is cluster capacity - assertEquals(0, appState.reviewRequestAndReleaseNodes().size()); - - // now do a node update (this doesn't touch the YARN engine; the node - // isn't really there) - NodeUpdatedOutcome outcome = addNewNode(); - assertEquals(cloneNodemap().size(), NODES + 1); - assertTrue(outcome.clusterChanged); - // no active calls to empty - assertTrue(outcome.operations.isEmpty()); - assertEquals(1, appState.reviewRequestAndReleaseNodes().size()); - } - - protected AppState.NodeUpdatedOutcome addNewNode() { - return updateNodes(MockFactory.INSTANCE.newNodeReport("4", NodeState - .RUNNING, "gpu")); - } - - //@Test - public void testClusterSizeChangesDuringRequestSequence() throws Throwable { - RoleStatus aaRole = getAaRole(); - describe("Change the cluster size where the cluster size changes during " + - "a test sequence."); - aaRole.setDesired(NODES + 1); - appState.reviewRequestAndReleaseNodes(); - assertTrue(aaRole.isAARequestOutstanding()); - assertEquals(NODES, aaRole.getAAPending()); - NodeUpdatedOutcome outcome = addNewNode(); - assertTrue(outcome.clusterChanged); - // one call to cancel - assertEquals(1, outcome.operations.size()); - // and on a review, one more to rebuild - assertEquals(1, appState.reviewRequestAndReleaseNodes().size()); - } - - //@Test - public void testBindingInfoMustHaveNodeMap() throws Throwable { - AppStateBindingInfo bindingInfo = buildBindingInfo(); - bindingInfo.nodeReports = null; - try { - MockAppState state = new MockAppState(bindingInfo); - fail("Expected an exception, got " + state); - } catch (IllegalArgumentException expected) { - } - } - - //@Test - public void testAMRestart() throws Throwable { - int desiredAA = 3; - getAaRole().setDesired(desiredAA); - List instances = createAndStartNodes(); - List containers = new ArrayList<>(); - for (RoleInstance instance : instances) { - containers.add(instance.container); - } - - // now destroy the app state - AppStateBindingInfo bindingInfo = buildBindingInfo(); - bindingInfo.application = factory.newApplication(0, 0, desiredAA).name( - getValidTestName()); - bindingInfo.application.getComponent(ROLE2) - .getConfiguration().setProperty(COMPONENT_PLACEMENT_POLICY, - Integer.toString(PlacementPolicy.ANTI_AFFINITY_REQUIRED)); - bindingInfo.liveContainers = containers; - appState = new MockAppState(bindingInfo); - - RoleStatus aaRole = lookupRole(AAROLE_2.name); - RoleStatus gpuRole = lookupRole(MockFactory.AAROLE_1_GPU.name); - appState.reviewRequestAndReleaseNodes(); - assertTrue(aaRole.isAntiAffinePlacement()); - assertTrue(aaRole.isAARequestOutstanding()); - - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateContainerFailure.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateContainerFailure.java deleted file mode 100644 index 41ff0fa..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateContainerFailure.java +++ /dev/null @@ -1,387 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.api.resource.Application; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.exceptions.TriggerClusterTeardownException; -import org.apache.slider.server.appmaster.actions.ResetFailureWindow; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockAM; -import org.apache.slider.server.appmaster.model.mock.MockAppState; -import org.apache.slider.server.appmaster.model.mock.MockRMOperationHandler; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.model.mock.MockYarnEngine; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.AppStateBindingInfo; -import org.apache.slider.server.appmaster.state.ContainerOutcome; -import org.apache.slider.server.appmaster.state.NodeEntry; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.RoleHistory; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; - -/** - * Test that if you have >1 role, the right roles are chosen for release. - */ -public class TestMockAppStateContainerFailure extends BaseMockAppStateTest - implements MockRoles { - private static final Logger LOG = - LoggerFactory.getLogger(TestMockAppStateContainerFailure.class); - - private MockRMOperationHandler operationHandler = new - MockRMOperationHandler(); - private MockAM mockAM = new MockAM(); - - @Override - public String getTestName() { - return "TestMockAppStateContainerFailure"; - } - - /** - * Small cluster with multiple containers per node, - * to guarantee many container allocations on each node. - * @return - */ - @Override - public MockYarnEngine createYarnEngine() { - return new MockYarnEngine(4, 8000); - } - - @Override - public Application buildApplication() { - Application application = super.buildApplication(); - application.getConfiguration().setProperty( - ResourceKeys.CONTAINER_FAILURE_THRESHOLD, "10"); - return application; - } - - //@Test - public void testShortLivedFail() throws Throwable { - - getRole0Status().setDesired(1); - List instances = createAndStartNodes(); - assertEquals(1, instances.size()); - - RoleInstance instance = instances.get(0); - long created = instance.createTime; - long started = instance.startTime; - assertTrue(created > 0); - assertTrue(started >= created); - List ids = extractContainerIds(instances, ROLE0); - - ContainerId cid = ids.get(0); - assertTrue(appState.isShortLived(instance)); - AppState.NodeCompletionResult result = appState.onCompletedContainer( - containerStatus(cid, 1)); - assertNotNull(result.roleInstance); - assertTrue(result.containerFailed); - RoleStatus status = getRole0Status(); - assertEquals(1, status.getFailed()); -// assertEquals(1, status.getStartFailed()); - - //view the world - appState.getRoleHistory().dump(); - List queue = appState.getRoleHistory().cloneRecentNodeList( - getRole0Status().getKey()); - assertEquals(0, queue.size()); - - } - - //@Test - public void testLongLivedFail() throws Throwable { - - getRole0Status().setDesired(1); - List instances = createAndStartNodes(); - assertEquals(1, instances.size()); - - RoleInstance instance = instances.get(0); - instance.startTime = System.currentTimeMillis() - 60 * 60 * 1000; - assertFalse(appState.isShortLived(instance)); - List ids = extractContainerIds(instances, ROLE0); - - ContainerId cid = ids.get(0); - AppState.NodeCompletionResult result = appState.onCompletedContainer( - containerStatus(cid, 1)); - assertNotNull(result.roleInstance); - assertTrue(result.containerFailed); - RoleStatus status = getRole0Status(); - assertEquals(1, status.getFailed()); -// assertEquals(0, status.getStartFailed()); - - //view the world - appState.getRoleHistory().dump(); - List queue = appState.getRoleHistory().cloneRecentNodeList( - getRole0Status().getKey()); - assertEquals(1, queue.size()); - - } - - //@Test - public void testNodeStartFailure() throws Throwable { - - getRole0Status().setDesired(1); - List instances = createAndSubmitNodes(); - assertEquals(1, instances.size()); - - RoleInstance instance = instances.get(0); - - List ids = extractContainerIds(instances, ROLE0); - - ContainerId cid = ids.get(0); - appState.onNodeManagerContainerStartFailed(cid, new SliderException( - "oops")); - RoleStatus status = getRole0Status(); - assertEquals(1, status.getFailed()); -// assertEquals(1, status.getStartFailed()); - - - RoleHistory history = appState.getRoleHistory(); - history.dump(); - List queue = history.cloneRecentNodeList(getRole0Status() - .getKey()); - assertEquals(0, queue.size()); - - NodeInstance ni = history.getOrCreateNodeInstance(instance.container); - NodeEntry re = ni.get(getRole0Status().getKey()); - assertEquals(1, re.getFailed()); - assertEquals(1, re.getStartFailed()); - } - - //@Test - public void testRecurrentStartupFailure() throws Throwable { - - getRole0Status().setDesired(1); - try { - for (int i = 0; i< 100; i++) { - List instances = createAndSubmitNodes(); - assertEquals(1, instances.size()); - - List ids = extractContainerIds(instances, ROLE0); - - ContainerId cid = ids.get(0); - LOG.info("{} instance {} {}", i, instances.get(0), cid); - assertNotNull(cid); - appState.onNodeManagerContainerStartFailed(cid, - new SliderException("failure #" + i)); - AppState.NodeCompletionResult result = appState.onCompletedContainer( - containerStatus(cid)); - assertTrue(result.containerFailed); - } - fail("Cluster did not fail from too many startup failures"); - } catch (TriggerClusterTeardownException teardown) { - LOG.info("Exception {} : {}", teardown.getExitCode(), teardown); - } - } - - //@Test - public void testRecurrentStartupFailureWithUnlimitedFailures() throws - Throwable { - // Update instance definition to allow containers to fail any number of - // times - AppStateBindingInfo bindingInfo = buildBindingInfo(); - bindingInfo.application.getComponent(ROLE0).getConfiguration().setProperty( - ResourceKeys.CONTAINER_FAILURE_THRESHOLD, "0"); - appState = new MockAppState(bindingInfo); - - getRole0Status().setDesired(1); - try { - for (int i = 0; i < 100; i++) { - List instances = createAndSubmitNodes(); - assertEquals(1, instances.size()); - - List ids = extractContainerIds(instances, ROLE0); - - ContainerId cid = ids.get(0); - LOG.info("{} instance {} {}", i, instances.get(0), cid); - assertNotNull(cid); - appState.onNodeManagerContainerStartFailed(cid, - new SliderException("failure #" + i)); - AppState.NodeCompletionResult result = appState.onCompletedContainer( - containerStatus(cid)); - assertTrue(result.containerFailed); - } - } catch (TriggerClusterTeardownException teardown) { - LOG.info("Exception {} : {}", teardown.getExitCode(), teardown); - fail("Cluster failed despite " + ResourceKeys - .CONTAINER_FAILURE_THRESHOLD + " = 0"); - } - } - - //@Test - public void testRoleStatusFailureWindow() throws Throwable { - - ResetFailureWindow resetter = new ResetFailureWindow(operationHandler); - - // initial reset - resetter.execute(mockAM, null, appState); - - getRole0Status().setDesired(1); - for (int i = 0; i < 100; i++) { - resetter.execute(mockAM, null, appState); - List instances = createAndSubmitNodes(); - assertEquals(1, instances.size()); - - List ids = extractContainerIds(instances, ROLE0); - - ContainerId cid = ids.get(0); - LOG.info("{} instance {} {}", i, instances.get(0), cid); - assertNotNull(cid); - appState.onNodeManagerContainerStartFailed( - cid, - new SliderException("failure #" + i)); - AppState.NodeCompletionResult result = appState.onCompletedContainer( - containerStatus(cid)); - assertTrue(result.containerFailed); - } - } - - //@Test - public void testRoleStatusFailed() throws Throwable { - RoleStatus status = getRole0Status(); - // limits exceeded - appState.incFailedContainers(status, ContainerOutcome.Failed); - assertEquals(1, status.getFailed()); - assertEquals(1L, status.getFailedRecently()); - assertEquals(0L, status.getLimitsExceeded()); - assertEquals(0L, status.getPreempted()); - assertEquals(0L, status.getDiskFailed()); - - ResetFailureWindow resetter = new ResetFailureWindow(operationHandler); - resetter.execute(mockAM, null, appState); - assertEquals(1, status.getFailed()); - assertEquals(0L, status.getFailedRecently()); - } - - //@Test - public void testRoleStatusFailedLimitsExceeded() throws Throwable { - RoleStatus status = getRole0Status(); - // limits exceeded - appState.incFailedContainers(status, ContainerOutcome - .Failed_limits_exceeded); - assertEquals(1, status.getFailed()); - assertEquals(1L, status.getFailedRecently()); - assertEquals(1L, status.getLimitsExceeded()); - assertEquals(0L, status.getPreempted()); - assertEquals(0L, status.getDiskFailed()); - - ResetFailureWindow resetter = new ResetFailureWindow(operationHandler); - resetter.execute(mockAM, null, appState); - assertEquals(1, status.getFailed()); - assertEquals(0L, status.getFailedRecently()); - assertEquals(1L, status.getLimitsExceeded()); - } - - - //@Test - public void testRoleStatusFailedPrempted() throws Throwable { - RoleStatus status = getRole0Status(); - // limits exceeded - appState.incFailedContainers(status, ContainerOutcome.Preempted); - assertEquals(0, status.getFailed()); - assertEquals(1L, status.getPreempted()); - assertEquals(0L, status.getFailedRecently()); - assertEquals(0L, status.getDiskFailed()); - - ResetFailureWindow resetter = new ResetFailureWindow(operationHandler); - resetter.execute(mockAM, null, appState); - assertEquals(1L, status.getPreempted()); - } - - - //@Test - public void testRoleStatusFailedNode() throws Throwable { - RoleStatus status = getRole0Status(); - // limits exceeded - appState.incFailedContainers(status, ContainerOutcome.Disk_failure); - assertEquals(1, status.getFailed()); - assertEquals(0L, status.getFailedRecently()); - assertEquals(0L, status.getLimitsExceeded()); - assertEquals(0L, status.getPreempted()); - assertEquals(1L, status.getDiskFailed()); - } - - //@Test - public void testNodeEntryCompleted() throws Throwable { - NodeEntry nodeEntry = new NodeEntry(1); - nodeEntry.containerCompleted(true, ContainerOutcome.Completed); - assertEquals(0, nodeEntry.getFailed()); - assertEquals(0, nodeEntry.getFailedRecently()); - assertEquals(0, nodeEntry.getStartFailed()); - assertEquals(0, nodeEntry.getPreempted()); - assertEquals(0, nodeEntry.getActive()); - assertTrue(nodeEntry.isAvailable()); - } - - //@Test - public void testNodeEntryFailed() throws Throwable { - NodeEntry nodeEntry = new NodeEntry(1); - nodeEntry.containerCompleted(false, ContainerOutcome.Failed); - assertEquals(1, nodeEntry.getFailed()); - assertEquals(1, nodeEntry.getFailedRecently()); - assertEquals(0, nodeEntry.getStartFailed()); - assertEquals(0, nodeEntry.getPreempted()); - assertEquals(0, nodeEntry.getActive()); - assertTrue(nodeEntry.isAvailable()); - nodeEntry.resetFailedRecently(); - assertEquals(1, nodeEntry.getFailed()); - assertEquals(0, nodeEntry.getFailedRecently()); - } - - //@Test - public void testNodeEntryLimitsExceeded() throws Throwable { - NodeEntry nodeEntry = new NodeEntry(1); - nodeEntry.containerCompleted(false, ContainerOutcome - .Failed_limits_exceeded); - assertEquals(0, nodeEntry.getFailed()); - assertEquals(0, nodeEntry.getFailedRecently()); - assertEquals(0, nodeEntry.getStartFailed()); - assertEquals(0, nodeEntry.getPreempted()); - } - - //@Test - public void testNodeEntryPreempted() throws Throwable { - NodeEntry nodeEntry = new NodeEntry(1); - nodeEntry.containerCompleted(false, ContainerOutcome.Preempted); - assertEquals(0, nodeEntry.getFailed()); - assertEquals(0, nodeEntry.getFailedRecently()); - assertEquals(0, nodeEntry.getStartFailed()); - assertEquals(1, nodeEntry.getPreempted()); - } - - //@Test - public void testNodeEntryNodeFailure() throws Throwable { - NodeEntry nodeEntry = new NodeEntry(1); - nodeEntry.containerCompleted(false, ContainerOutcome.Disk_failure); - assertEquals(1, nodeEntry.getFailed()); - assertEquals(1, nodeEntry.getFailedRecently()); - assertEquals(0, nodeEntry.getStartFailed()); - assertEquals(0, nodeEntry.getPreempted()); - } - - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDependencies.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDependencies.java deleted file mode 100644 index cbef2be..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDependencies.java +++ /dev/null @@ -1,163 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.service.compinstance.ComponentInstance; -import org.apache.slider.api.types.ApplicationLivenessInformation; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.state.ContainerAssignment; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.apache.slider.server.servicemonitor.ProbeStatus; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -/** - * Test for postponing container requests until dependencies are ready. - */ -public class TestMockAppStateDependencies extends BaseMockAppStateTest - implements MockRoles { - - private org.apache.slider.server.servicemonitor.Probe successProbe = - new org.apache.slider.server.servicemonitor.Probe("success", null) { - @Override - public ProbeStatus ping(ComponentInstance roleInstance) { - ProbeStatus status = new ProbeStatus(); - status.succeed(this); - return status; - } - }; - - private org.apache.slider.server.servicemonitor.Probe failureProbe = - new org.apache.slider.server.servicemonitor.Probe("failure", null) { - @Override - public ProbeStatus ping(ComponentInstance roleInstance) { - ProbeStatus status = new ProbeStatus(); - status.fail(this, new Exception()); - return status; - } - }; - - @Override - public String getTestName() { - return "TestMockAppStateDependencies"; - } - - //@Test - public void testDependencies() throws Throwable { - RoleStatus role0Status = getRole0Status(); - RoleStatus role1Status = getRole1Status(); - - // set desired instances for role0 to 1 - role0Status.setDesired(1); - // set probe for role0 to use a ping that will always succeed - role0Status.getProviderRole().probe = successProbe; - - // set desired instances for role1 to 1 - role1Status.setDesired(1); - // set role0 as a dependency of role1 - role1Status.getProviderRole().component.setDependencies(Collections - .singletonList(ROLE0)); - - // role0 has no dependencies, so its dependencies are ready - assertTrue(appState.areDependenciesReady(role0Status)); - // role1 dependency (role0) is not ready yet - assertFalse(appState.areDependenciesReady(role1Status)); - // start the single requested instance for role0 - review(ROLE0, 2); - - // role0 is still not ready because a ping has not been issued - assertFalse(appState.areDependenciesReady(role1Status)); - // issue pings - appState.monitorComponentInstances(); - // now role0 is ready - assertTrue(appState.areDependenciesReady(role1Status)); - // increase the desired containers for role0 - role0Status.setDesired(2); - // role0 is no longer ready - assertFalse(appState.areDependenciesReady(role1Status)); - // start a second instance for role0 - review(ROLE0, 2); - - // role0 is not ready because ping has not been issued for the new instance - assertFalse(appState.areDependenciesReady(role1Status)); - // issue pings - appState.monitorComponentInstances(); - // role0 is ready - assertTrue(appState.areDependenciesReady(role1Status)); - - // set probe for role0 to use a ping that will always fail - role0Status.getProviderRole().probe = failureProbe; - // issue pings - appState.monitorComponentInstances(); - // role0 is not ready (failure probe works) - assertFalse(appState.areDependenciesReady(role1Status)); - // set probe for role0 to use a ping that will always succeed - role0Status.getProviderRole().probe = successProbe; - // issue pings - appState.monitorComponentInstances(); - // role0 is ready - assertTrue(appState.areDependenciesReady(role1Status)); - - // now role1 instances can be started - review(ROLE1, 1); - } - - public void review(String expectedRole, int outstanding) throws Exception { - List ops = appState.reviewRequestAndReleaseNodes(); - - // expect one request in the list - assertEquals(1, ops.size()); - // and in a liveness check, expected outstanding - ApplicationLivenessInformation liveness = - appState.getApplicationLivenessInformation(); - assertEquals(outstanding, liveness.requestsOutstanding); - assertFalse(liveness.allRequestsSatisfied); - - // record container allocated and verify it has the expected role - List allocations = engine.execute(ops); - List assignments = new ArrayList<>(); - List releases = new ArrayList<>(); - appState.onContainersAllocated(allocations, assignments, releases); - assertEquals(1, assignments.size()); - ContainerAssignment assigned = assignments.get(0); - Container target = assigned.container; - RoleInstance ri = roleInstance(assigned); - assertEquals(expectedRole, ri.role); - - // one fewer request outstanding - liveness = appState.getApplicationLivenessInformation(); - assertEquals(outstanding - 1, liveness.requestsOutstanding); - - // record container start submitted - appState.containerStartSubmitted(target, ri); - - // additional review results in no additional requests - ops = appState.reviewRequestAndReleaseNodes(); - assertTrue(ops.isEmpty()); - - // record container start - appState.innerOnNodeManagerContainerStarted(target.getId()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicHistory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicHistory.java deleted file mode 100644 index 76f3f37..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicHistory.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.providers.PlacementPolicy; -import org.apache.slider.providers.ProviderRole; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockRoleHistory; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.model.mock.MockYarnEngine; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.ContainerRequestOperation; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.NodeEntry; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.RoleHistory; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -/** - * Test that if you have >1 role, the right roles are chosen for release. - */ -public class TestMockAppStateDynamicHistory extends BaseMockAppStateTest - implements MockRoles { - private static final Logger LOG = - LoggerFactory.getLogger(TestMockAppStateDynamicHistory.class); - - /** - * Small cluster with multiple containers per node, - * to guarantee many container allocations on each node. - * @return - */ - @Override - public MockYarnEngine createYarnEngine() { - return new MockYarnEngine(8, 1); - } - - // TODO does not support adding new components dynamically - public void testDynamicRoleHistory() throws Throwable { - - String dynamic = "dynamicRole"; - long desired = 1; - int placementPolicy = PlacementPolicy.DEFAULT; - // snapshot and patch existing spec - Application application = appState.getClusterStatus(); - Component component = new Component().name(dynamic).numberOfContainers( - desired); - component.getConfiguration().setProperty(ResourceKeys - .COMPONENT_PLACEMENT_POLICY, "" + placementPolicy); - application.getComponents().add(component); - - appState.updateComponents( - Collections.singletonMap(dynamic, desired)); - - // now look at the role map - assertNotNull(appState.getRoleMap().get(dynamic)); - ProviderRole mappedRole = appState.getRoleMap().get(dynamic); - int rolePriority = mappedRole.id; - - Map priorityMap = appState.getRolePriorityMap(); - assertEquals(priorityMap.size(), 4); - ProviderRole dynamicProviderRole = priorityMap.get(rolePriority); - assertNotNull(dynamicProviderRole); - assertEquals(dynamicProviderRole.id, rolePriority); - - assertNotNull(appState.getRoleStatusMap().get(rolePriority)); - RoleStatus dynamicRoleStatus = - appState.getRoleStatusMap().get(rolePriority); - assertEquals(dynamicRoleStatus.getDesired(), desired); - - - // before allocating the nodes, fill up the capacity of some of the - // hosts - engine.getAllocator().nextIndex(); - - int targetNode = 2; - assertEquals(targetNode, engine.getAllocator().nextIndex()); - String targetHostname = engine.getCluster().nodeAt(targetNode) - .getHostname(); - - // clock is set to a small value - appState.setTime(100000); - - // allocate the nodes - List actions = appState.reviewRequestAndReleaseNodes(); - assertEquals(1, actions.size()); - ContainerRequestOperation action0 = (ContainerRequestOperation)actions - .get(0); - - ContainerRequest request = action0.getRequest(); - assertTrue(SliderUtils.isEmpty(request.getNodes())); - - List released = new ArrayList<>(); - List allocations = submitOperations(actions, released); - processSubmissionOperations(allocations, new ArrayList<>(), released); - assertEquals(1, allocations.size()); - RoleInstance ri = allocations.get(0); - - assertEquals(ri.role, dynamic); - assertEquals(ri.roleId, rolePriority); - assertEquals(ri.host, targetHostname); - - // now look at the role history - - RoleHistory roleHistory = appState.getRoleHistory(); - List activeNodes = roleHistory.listActiveNodes( - rolePriority); - assertEquals(activeNodes.size(), 1); - NodeInstance activeNode = activeNodes.get(0); - assertNotNull(activeNode.get(rolePriority)); - NodeEntry entry8 = activeNode.get(rolePriority); - assertEquals(entry8.getActive(), 1); - - assertEquals(activeNode.hostname, targetHostname); - - NodeInstance activeNodeInstance = - roleHistory.getOrCreateNodeInstance(ri.container); - - assertEquals(activeNode, activeNodeInstance); - NodeEntry entry = activeNodeInstance.get(rolePriority); - assertNotNull(entry); - assertTrue(entry.getActive() > 0); - assertTrue(entry.getLive() > 0); - - - // now trigger a termination event on that role - - // increment time for a long-lived failure event - appState.incTime(100000); - - LOG.debug("Triggering failure"); - ContainerId cid = ri.getContainerId(); - AppState.NodeCompletionResult result = appState.onCompletedContainer( - containerStatus(cid, 1)); - assertEquals(result.roleInstance, ri); - assertTrue(result.containerFailed); - - roleHistory.dump(); - // values should have changed - assertEquals(1, entry.getFailed()); - assertEquals(0, entry.getStartFailed()); - assertEquals(0, entry.getActive()); - assertEquals(0, entry.getLive()); - - - List nodesForRoleId = - roleHistory.getRecentNodesForRoleId(rolePriority); - assertNotNull(nodesForRoleId); - - // make sure new nodes will default to a different host in the engine - assertTrue(targetNode < engine.getAllocator().nextIndex()); - - actions = appState.reviewRequestAndReleaseNodes(); - assertEquals(1, actions.size()); - ContainerRequestOperation action1 = (ContainerRequestOperation) actions - .get(0); - ContainerRequest request1 = action1.getRequest(); - assertTrue(SliderUtils.isNotEmpty(request1.getNodes())); - } - - //@Test(expected = BadConfigException.class) - public void testRoleHistoryRoleAdditions() throws Throwable { - MockRoleHistory roleHistory = new MockRoleHistory(new ArrayList<>()); - roleHistory.addNewRole(new RoleStatus(new ProviderRole("one", 1))); - roleHistory.addNewRole(new RoleStatus(new ProviderRole("two", 1))); - roleHistory.dump(); - } - - //@Test(expected = BadConfigException.class) - public void testRoleHistoryRoleStartupConflict() throws Throwable { - MockRoleHistory roleHistory = new MockRoleHistory(Arrays.asList( - new ProviderRole("one", 1), new ProviderRole("two", 1) - )); - roleHistory.dump(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.java deleted file mode 100644 index 5669d2d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateDynamicRoles.java +++ /dev/null @@ -1,243 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.providers.PlacementPolicy; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.model.mock.MockYarnEngine; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.ContainerRequestOperation; -import org.apache.slider.server.appmaster.state.AppState.NodeCompletionResult; -import org.apache.slider.server.appmaster.state.ContainerPriority; -import org.apache.slider.server.appmaster.state.RoleHistoryUtils; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; - -import static org.apache.slider.server.appmaster.model.mock.MockFactory.NODE_FAILURE_THRESHOLD; - -/** - * Test that if you have >1 role, the right roles are chosen for release. - */ -public class TestMockAppStateDynamicRoles extends BaseMockAppStateTest - implements MockRoles { - private static final Logger LOG = - LoggerFactory.getLogger(TestMockAppStateDynamicRoles.class); - private static final String ROLE4 = "4"; - private static final String ROLE5 = "5"; - - @Override - public String getTestName() { - return "TestMockAppStateDynamicRoles"; - } - - /** - * Small cluster with multiple containers per node, - * to guarantee many container allocations on each node. - * @return - */ - @Override - public MockYarnEngine createYarnEngine() { - return new MockYarnEngine(8, 2); - } - - @Override - public Application buildApplication() { - Application application = super.buildApplication(); - - Component component = new Component().name(ROLE4).numberOfContainers(1L); - component.getConfiguration().setProperty(ResourceKeys - .NODE_FAILURE_THRESHOLD, Integer.toString(3)); - application.getComponents().add(component); - - component = new Component().name(ROLE5).numberOfContainers(1L); - component.getConfiguration().setProperty(ResourceKeys - .COMPONENT_PLACEMENT_POLICY, Integer.toString(PlacementPolicy.STRICT)); - application.getComponents().add(component); - - return application; - } - - //@Test - public void testAllocateReleaseRealloc() throws Throwable { - - createAndStartNodes(); - appState.reviewRequestAndReleaseNodes(); - appState.getRoleHistory().dump(); - } - - /** - * Find all allocations for a specific role. - * @param role role Id/priority - * @param actions source list - * @return found list - */ - List findAllocationsForRole(int role, - List actions) { - List ops = new ArrayList<>(); - for (AbstractRMOperation op : actions) { - if (op instanceof ContainerRequestOperation && role == - ContainerPriority.extractRole(((ContainerRequestOperation) op) - .getRequest().getPriority())) { - ops.add((ContainerRequestOperation) op); - } - } - return ops; - } - - //@Test - public void testStrictPlacementInitialRequest() throws Throwable { - LOG.info("Initial engine state = {}", engine); - List actions = appState.reviewRequestAndReleaseNodes(); - assertEquals(2, actions.size()); - - // neither have locality at this point - assertRelaxLocalityFlag(appState.lookupRoleStatus(ROLE4).getKey(), null, - true, actions); - assertRelaxLocalityFlag(appState.lookupRoleStatus(ROLE5).getKey(), null, - true, actions); - } - - //@Test - public void testPolicyPropagation() throws Throwable { - assertEquals(0, (appState.lookupRoleStatus(ROLE4).getPlacementPolicy() & - PlacementPolicy.STRICT)); - assertNotEquals(0, (appState.lookupRoleStatus(ROLE5).getPlacementPolicy() & - PlacementPolicy.STRICT)); - - } - - //@Test - public void testNodeFailureThresholdPropagation() throws Throwable { - assertEquals(3, appState.lookupRoleStatus(ROLE4).getNodeFailureThreshold()); - assertEquals(NODE_FAILURE_THRESHOLD, appState.lookupRoleStatus(ROLE5) - .getNodeFailureThreshold()); - } - - //@Test - public void testLaxPlacementSecondRequestRole4() throws Throwable { - LOG.info("Initial engine state = {}", engine); - RoleStatus role4 = appState.lookupRoleStatus(ROLE4); - RoleStatus role5 = appState.lookupRoleStatus(ROLE5); - role4.setDesired(1); - role5.setDesired(0); - - List instances = createStartAndStopNodes(new ArrayList<>()); - assertEquals(1, instances.size()); - - int id = appState.lookupRoleStatus(ROLE4).getKey(); - RoleInstance instanceA = null; - for (RoleInstance instance : instances) { - if (instance.roleId == id) { - instanceA = instance; - } - } - assertNotNull(instanceA); - String hostname = RoleHistoryUtils.hostnameOf(instanceA.container); - - LOG.info("Allocated engine state = {}", engine); - assertEquals(1, engine.containerCount()); - - assertEquals(1, role4.getRunning()); - // shrinking cluster - - role4.setDesired(0); - appState.lookupRoleStatus(ROLE4).setDesired(0); - List completionResults = new ArrayList<>(); - createStartAndStopNodes(completionResults); - assertEquals(0, engine.containerCount()); - assertEquals(1, completionResults.size()); - - // expanding: expect hostnames now - role4.setDesired(1); - List actions = appState.reviewRequestAndReleaseNodes(); - assertEquals(1, actions.size()); - - ContainerRequestOperation cro = (ContainerRequestOperation) actions.get(0); - List nodes = cro.getRequest().getNodes(); - assertEquals(1, nodes.size()); - assertEquals(hostname, nodes.get(0)); - } - - //@Test - public void testStrictPlacementSecondRequestRole5() throws Throwable { - LOG.info("Initial engine state = {}", engine); - RoleStatus role4 = appState.lookupRoleStatus(ROLE4); - RoleStatus role5 = appState.lookupRoleStatus(ROLE5); - role4.setDesired(0); - role5.setDesired(1); - - List instances = createStartAndStopNodes(new ArrayList<>()); - assertEquals(1, instances.size()); - - int id = appState.lookupRoleStatus(ROLE5).getKey(); - RoleInstance instanceA = null; - for (RoleInstance instance : instances) { - if (instance.roleId == id) { - instanceA = instance; - } - } - assertNotNull(instanceA); - String hostname = RoleHistoryUtils.hostnameOf(instanceA.container); - - LOG.info("Allocated engine state = {}", engine); - assertEquals(1, engine.containerCount()); - - assertEquals(1, role5.getRunning()); - - // shrinking cluster - role5.setDesired(0); - List completionResults = new ArrayList<>(); - createStartAndStopNodes(completionResults); - assertEquals(0, engine.containerCount()); - assertEquals(1, completionResults.size()); - assertEquals(0, role5.getRunning()); - - role5.setDesired(1); - List actions = appState.reviewRequestAndReleaseNodes(); - assertEquals(1, actions.size()); - assertRelaxLocalityFlag(id, "", false, actions); - ContainerRequestOperation cro = (ContainerRequestOperation) actions.get(0); - List nodes = cro.getRequest().getNodes(); - assertEquals(1, nodes.size()); - assertEquals(hostname, nodes.get(0)); - } - - public void assertRelaxLocalityFlag( - int role, - String expectedHost, - boolean expectedRelaxFlag, - List actions) { - List requests = findAllocationsForRole( - role, actions); - assertEquals(1, requests.size()); - ContainerRequestOperation req = requests.get(0); - assertEquals(expectedRelaxFlag, req.getRequest().getRelaxLocality()); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexDynamicRoles.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexDynamicRoles.java deleted file mode 100644 index 3304da1..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexDynamicRoles.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.fs.Path; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.core.exceptions.SliderInternalStateException; -import org.apache.slider.core.exceptions.TriggerClusterTeardownException; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockAppState; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.model.mock.MockYarnEngine; -import org.apache.slider.server.appmaster.state.AppStateBindingInfo; -import org.apache.slider.server.appmaster.state.MostRecentContainerReleaseSelector; -import org.apache.slider.server.appmaster.state.RoleHistory; -import org.apache.slider.server.avro.LoadedRoleHistory; -import org.apache.slider.server.avro.RoleHistoryWriter; -import org.junit.Before; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.util.Collections; - -/** - * Test that if you have more than one role, the right roles are chosen for - * release. - */ -public class TestMockAppStateFlexDynamicRoles extends BaseMockAppStateTest - implements MockRoles { - private static final Logger LOG = - LoggerFactory.getLogger(TestMockAppStateFlexDynamicRoles.class); - - @Override - public String getTestName() { - return "TestMockAppStateFlexDynamicRoles"; - } - - /** - * Small cluster with multiple containers per node, - * to guarantee many container allocations on each node. - * @return - */ - @Override - public MockYarnEngine createYarnEngine() { - return new MockYarnEngine(4, 4); - } - - @Override - public AppStateBindingInfo buildBindingInfo() throws IOException { - AppStateBindingInfo bindingInfo = super.buildBindingInfo(); - bindingInfo.releaseSelector = new MostRecentContainerReleaseSelector(); - return bindingInfo; - } - - @Override - public Application buildApplication() { - Application application = super.buildApplication(); - Component component = new Component().name("dynamic-6") - .numberOfContainers(1L); - application.getComponents().add(component); - - return application; - } - - @Before - public void init() - throws TriggerClusterTeardownException, SliderInternalStateException { - createAndStartNodes(); - } - - // TODO does not support adding new components dynamically - public void testDynamicFlexAddRole() throws Throwable { - Application application = appState.getClusterStatus(); - Component component = new Component().name("dynamicAdd7") - .numberOfContainers(1L); - application.getComponents().add(component); - appState.updateComponents(Collections.singletonMap(component.getName(), - component.getNumberOfContainers())); - createAndStartNodes(); - appState.lookupRoleStatus("dynamicAdd7"); - } - - //@Test - public void testDynamicFlexDropRole() throws Throwable { - appState.updateComponents(Collections.singletonMap("dynamic-6", 0L)); - //status is retained for future - appState.lookupRoleStatus("dynamic-6"); - } - - - //@Test - public void testHistorySaveFlexLoad() throws Throwable { - Application application = appState.getClusterStatus(); - RoleHistory roleHistory = appState.getRoleHistory(); - Path history = roleHistory.saveHistory(0x0001); - RoleHistoryWriter historyWriter = new RoleHistoryWriter(); - Component component = new Component().name("HistorySaveFlexLoad") - .numberOfContainers(1L); - application.getComponents().add(component); - - appState.updateComponents(Collections.singletonMap(component.getName(), - component.getNumberOfContainers())); - createAndStartNodes(); - LoadedRoleHistory loadedRoleHistory = - historyWriter.read(fs, history); - assertEquals(0, appState.getRoleHistory().rebuild(loadedRoleHistory)); - } - - //@Test - public void testHistoryFlexSaveResetLoad() throws Throwable { - Application application = appState.getClusterStatus(); - Component component = new Component().name("HistoryFlexSaveLoad") - .numberOfContainers(1L); - application.getComponents().add(component); - - appState.updateComponents(Collections.singletonMap(component.getName(), - component.getNumberOfContainers())); - createAndStartNodes(); - RoleHistoryWriter historyWriter = new RoleHistoryWriter(); - RoleHistory roleHistory = appState.getRoleHistory(); - Path history = roleHistory.saveHistory(0x0002); - //now reset the app state - File historyWorkDir2 = new File("target/history" + getTestName() + - "-0002"); - Path historyPath2 = new Path(historyWorkDir2.toURI()); - appState = new MockAppState(); - AppStateBindingInfo binding2 = buildBindingInfo(); - binding2.application = factory.newApplication(0, 0, 0) - .name(getValidTestName()); - binding2.historyPath = historyPath2; - appState.buildInstance(binding2); - // on this read there won't be the right number of roles - LoadedRoleHistory loadedRoleHistory = historyWriter.read(fs, history); - assertEquals(0, appState.getRoleHistory().rebuild(loadedRoleHistory)); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexing.java deleted file mode 100644 index dbad599..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateFlexing.java +++ /dev/null @@ -1,201 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.types.ApplicationLivenessInformation; -import org.apache.slider.core.exceptions.TriggerClusterTeardownException; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.CancelSingleRequest; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.ContainerAssignment; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; - -/** - * Test app state flexing. - */ -public class TestMockAppStateFlexing extends BaseMockAppStateTest implements - MockRoles { - private static final Logger LOG = - LoggerFactory.getLogger(BaseMockAppStateTest.class); - - @Override - public String getTestName() { - return "TestMockAppStateFlexing"; - } - - //@Test - public void testFlexDuringLaunchPhase() throws Throwable { - - // ask for one instance of role0 - getRole0Status().setDesired(1); - - List ops = appState.reviewRequestAndReleaseNodes(); - - // at this point there's now one request in the list - assertEquals(1, ops.size()); - // and in a liveness check, one outstanding - ApplicationLivenessInformation liveness = - appState.getApplicationLivenessInformation(); - assertEquals(1, liveness.requestsOutstanding); - assertFalse(liveness.allRequestsSatisfied); - - List allocations = engine.execute(ops); - List assignments = new ArrayList<>(); - List releases = new ArrayList<>(); - appState.onContainersAllocated(allocations, assignments, releases); - assertEquals(1, assignments.size()); - ContainerAssignment assigned = assignments.get(0); - Container target = assigned.container; - RoleInstance ri = roleInstance(assigned); - - ops = appState.reviewRequestAndReleaseNodes(); - assertTrue(ops.isEmpty()); - - liveness = appState.getApplicationLivenessInformation(); - assertEquals(0, liveness.requestsOutstanding); - assertTrue(liveness.allRequestsSatisfied); - - //now this is the start point. - appState.containerStartSubmitted(target, ri); - - ops = appState.reviewRequestAndReleaseNodes(); - assertTrue(ops.isEmpty()); - - appState.innerOnNodeManagerContainerStarted(target.getId()); - } - - //@Test - public void testFlexBeforeAllocationPhase() throws Throwable { - getRole0Status().setDesired(1); - - List ops = appState.reviewRequestAndReleaseNodes(); - assertFalse(ops.isEmpty()); - - // second scan will find the first run outstanding, so not re-issue - // any more container requests - List ops2 = appState.reviewRequestAndReleaseNodes(); - assertTrue(ops2.isEmpty()); - - // and in a liveness check, one outstanding - ApplicationLivenessInformation liveness = appState - .getApplicationLivenessInformation(); - assertEquals(1, liveness.requestsOutstanding); - assertFalse(liveness.allRequestsSatisfied); - - appState.refreshClusterStatus(); - Application application = appState.getClusterStatus(); - // TODO cluster status returns liveness info -// assertEquals(1, cd.liveness.requestsOutstanding); - - } - - - //@Test - public void testFlexDownTwice() throws Throwable { - int r0 = 6; - int r1 = 0; - int r2 = 0; - getRole0Status().setDesired(r0); - getRole1Status().setDesired(r1); - getRole2Status().setDesired(r2); - List instances = createAndStartNodes(); - - int clusterSize = r0 + r1 + r2; - assertEquals(instances.size(), clusterSize); - LOG.info("shrinking cluster"); - r0 = 4; - getRole0Status().setDesired(r0); - List completionResults = new ArrayList<>(); - instances = createStartAndStopNodes(completionResults); - assertEquals(0, instances.size()); - // assert two nodes were released - assertEquals(2, completionResults.size()); - - // no-op review - completionResults = new ArrayList<>(); - instances = createStartAndStopNodes(completionResults); - assertEquals(0, instances.size()); - // assert two nodes were released - assertEquals(0, completionResults.size()); - - - // now shrink again - getRole0Status().setDesired(1); - completionResults = new ArrayList<>(); - instances = createStartAndStopNodes(completionResults); - assertEquals(0, instances.size()); - // assert two nodes were released - assertEquals(3, completionResults.size()); - - } - - //@Test - public void testFlexNegative() throws Throwable { - int r0 = 6; - int r1 = 0; - int r2 = 0; - getRole0Status().setDesired(r0); - getRole1Status().setDesired(r1); - getRole2Status().setDesired(r2); - List instances = createAndStartNodes(); - - int clusterSize = r0 + r1 + r2; - assertEquals(instances.size(), clusterSize); - LOG.info("shrinking cluster"); - getRole0Status().setDesired(-2); - List completionResults = new ArrayList<>(); - try { - createStartAndStopNodes(completionResults); - fail("expected an exception"); - } catch (TriggerClusterTeardownException e) { - } - - } - - //@Test - public void testCancelWithRequestsOutstanding() throws Throwable { - // flex cluster size before the original set were allocated - - - getRole0Status().setDesired(6); - // build the ops - List ops = appState.reviewRequestAndReleaseNodes(); - // here the data structures exist - - // go down - getRole0Status().setDesired(3); - List ops2 = appState.reviewRequestAndReleaseNodes(); - assertEquals(3, ops2.size()); - for (AbstractRMOperation op : ops2) { - assertTrue(op instanceof CancelSingleRequest); - } - - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRMOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRMOperations.java deleted file mode 100644 index 8686479..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRMOperations.java +++ /dev/null @@ -1,430 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockRMOperationHandler; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.model.mock.MockYarnEngine; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.CancelSingleRequest; -import org.apache.slider.server.appmaster.operations.ContainerReleaseOperation; -import org.apache.slider.server.appmaster.operations.ContainerRequestOperation; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.ContainerAssignment; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; - -import static org.apache.slider.server.appmaster.state.ContainerPriority.buildPriority; -import static org.apache.slider.server.appmaster.state.ContainerPriority.extractRole; - -/** - * Test app state RM operations. - */ -public class TestMockAppStateRMOperations extends BaseMockAppStateTest - implements MockRoles { - private static final Logger LOG = - LoggerFactory.getLogger(BaseMockAppStateTest.class); - - @Override - public String getTestName() { - return "TestMockAppStateRMOperations"; - } - - //@Test - public void testPriorityOnly() throws Throwable { - assertEquals(5, extractRole(buildPriority(5, false))); - } - - //@Test - public void testPriorityRoundTrip() throws Throwable { - assertEquals(5, extractRole(buildPriority(5, false))); - } - - //@Test - public void testPriorityRoundTripWithRequest() throws Throwable { - int priority = buildPriority(5, false); - assertEquals(5, extractRole(priority)); - } - - //@Test - public void testMockAddOp() throws Throwable { - getRole0Status().setDesired(1); - List ops = appState.reviewRequestAndReleaseNodes(); - assertListLength(ops, 1); - ContainerRequestOperation operation = (ContainerRequestOperation)ops.get(0); - int priority = operation.getRequest().getPriority().getPriority(); - assertEquals(extractRole(priority), getRole0Status().getKey()); - MockRMOperationHandler handler = new MockRMOperationHandler(); - handler.execute(ops); - - AbstractRMOperation op = handler.getFirstOp(); - assertTrue(op instanceof ContainerRequestOperation); - } - - /** - * Test of a flex up and down op which verifies that outstanding - * requests are cancelled first. - *
    - *
  1. request 5 nodes, assert 5 request made
  2. - *
  3. allocate 1 of them
  4. - *
  5. flex cluster size to 3
  6. - *
  7. assert this generates 2 cancel requests
  8. - *
- */ - //@Test - public void testRequestThenCancelOps() throws Throwable { - RoleStatus role0 = getRole0Status(); - role0.setDesired(5); - List ops = appState.reviewRequestAndReleaseNodes(); - assertListLength(ops, 5); - // now 5 outstanding requests. - assertEquals(5, role0.getRequested()); - - // allocate one - List processed = new ArrayList<>(); - processed.add(ops.get(0)); - List released = new ArrayList<>(); - List completionResults = new ArrayList<>(); - submitOperations(processed, released); - List instances = createAndSubmitNodes(released); - processSubmissionOperations(instances, completionResults, released); - - - // four outstanding - assertEquals(4, role0.getRequested()); - - // flex cluster to 3 - role0.setDesired(3); - ops = appState.reviewRequestAndReleaseNodes(); - - // expect two cancel operation from review - assertListLength(ops, 2); - for (AbstractRMOperation op : ops) { - assertTrue(op instanceof CancelSingleRequest); - } - - MockRMOperationHandler handler = new MockRMOperationHandler(); - handler.setAvailableToCancel(4); - handler.execute(ops); - assertEquals(2, handler.getAvailableToCancel()); - assertEquals(2, role0.getRequested()); - - // flex down one more - role0.setDesired(2); - ops = appState.reviewRequestAndReleaseNodes(); - assertListLength(ops, 1); - for (AbstractRMOperation op : ops) { - assertTrue(op instanceof CancelSingleRequest); - } - handler.execute(ops); - assertEquals(1, handler.getAvailableToCancel()); - assertEquals(1, role0.getRequested()); - } - - //@Test - public void testCancelNoActualContainers() throws Throwable { - RoleStatus role0 = getRole0Status(); - role0.setDesired(5); - List ops = appState.reviewRequestAndReleaseNodes(); - assertListLength(ops, 5); - // now 5 outstanding requests. - assertEquals(5, role0.getRequested()); - role0.setDesired(0); - ops = appState.reviewRequestAndReleaseNodes(); - assertListLength(ops, 5); - - } - - - //@Test - public void testFlexDownOutstandingRequests() throws Throwable { - // engine only has two nodes, so > 2 will be outstanding - engine = new MockYarnEngine(1, 2); - List ops; - // role: desired = 2, requested = 1, actual=1 - RoleStatus role0 = getRole0Status(); - role0.setDesired(4); - createAndSubmitNodes(); - - assertEquals(2, role0.getRequested()); - assertEquals(2, role0.getRunning()); - // there are now two outstanding, two actual - // Release 3 and verify that the two - // cancellations were combined with a release - role0.setDesired(1); - assertEquals(-3, role0.getDelta()); - ops = appState.reviewRequestAndReleaseNodes(); - assertListLength(ops, 3); - int numCancel = 0; - int numRelease = 0; - for (AbstractRMOperation op : ops) { - if (op instanceof CancelSingleRequest) { - numCancel++; - } - if (op instanceof ContainerReleaseOperation) { - numRelease++; - } - } - assertEquals(2, numCancel); - assertEquals(1, numRelease); - assertEquals(0, role0.getRequested()); - // TODO releasing? -// assertEquals(1, role0.getReleasing()); - } - - //@Test - public void testCancelAllOutstandingRequests() throws Throwable { - - // role: desired = 2, requested = 1, actual=1 - RoleStatus role0 = getRole0Status(); - role0.setDesired(2); - List ops; - ops = appState.reviewRequestAndReleaseNodes(); - int count = 0; - for (AbstractRMOperation op : ops) { - if (op instanceof ContainerRequestOperation) { - count++; - } - } - assertEquals(2, count); - - // there are now two outstanding, two actual - // Release 3 and verify that the two - // cancellations were combined with a release - role0.setDesired(0); - ops = appState.reviewRequestAndReleaseNodes(); - assertEquals(2, ops.size()); - - for (AbstractRMOperation op : ops) { - assertTrue(op instanceof CancelSingleRequest); - } - } - - - //@Test - public void testFlexUpOutstandingRequests() throws Throwable { - - List ops; - // role: desired = 2, requested = 1, actual=1 - RoleStatus role0 = getRole0Status(); - role0.setDesired(2); - appState.incRunningContainers(role0); - appState.incRequestedContainers(role0); - - // flex up 2 nodes, yet expect only one node to be requested, - // as the outstanding request is taken into account - role0.setDesired(4); - appState.incRequestedContainers(role0); - - assertEquals(1, role0.getRunning()); - assertEquals(2, role0.getRequested()); - assertEquals(3, role0.getActualAndRequested()); - assertEquals(1, role0.getDelta()); - ops = appState.reviewRequestAndReleaseNodes(); - assertListLength(ops, 1); - assertTrue(ops.get(0) instanceof ContainerRequestOperation); - assertEquals(3, role0.getRequested()); - } - - //@Test - public void testFlexUpNoSpace() throws Throwable { - // engine only has two nodes, so > 2 will be outstanding - engine = new MockYarnEngine(1, 2); - // role: desired = 2, requested = 1, actual=1 - RoleStatus role0 = getRole0Status(); - role0.setDesired(4); - createAndSubmitNodes(); - - assertEquals(2, role0.getRequested()); - assertEquals(2, role0.getRunning()); - role0.setDesired(8); - assertEquals(4, role0.getDelta()); - createAndSubmitNodes(); - assertEquals(6, role0.getRequested()); - } - - - //@Test - public void testAllocateReleaseOp() throws Throwable { - getRole0Status().setDesired(1); - - List ops = appState.reviewRequestAndReleaseNodes(); - ContainerRequestOperation operation = (ContainerRequestOperation)ops.get(0); - AMRMClient.ContainerRequest request = operation.getRequest(); - Container cont = engine.allocateContainer(request); - List allocated = new ArrayList<>(); - allocated.add(cont); - List assignments = new ArrayList<>(); - List operations = new ArrayList<>(); - appState.onContainersAllocated(allocated, assignments, operations); - - assertListLength(ops, 1); - assertListLength(assignments, 1); - ContainerAssignment assigned = assignments.get(0); - Container target = assigned.container; - assertEquals(target.getId(), cont.getId()); - int roleId = assigned.role.getPriority(); - assertEquals(roleId, extractRole(request.getPriority())); - assertEquals(assigned.role.getName(), ROLE0); - RoleInstance ri = roleInstance(assigned); - //tell the app it arrived - appState.containerStartSubmitted(target, ri); - appState.innerOnNodeManagerContainerStarted(target.getId()); - assertEquals(1, getRole0Status().getRunning()); - - //now release it by changing the role status - getRole0Status().setDesired(0); - ops = appState.reviewRequestAndReleaseNodes(); - assertListLength(ops, 1); - - assertTrue(ops.get(0) instanceof ContainerReleaseOperation); - ContainerReleaseOperation release = (ContainerReleaseOperation) ops.get(0); - assertEquals(release.getContainerId(), cont.getId()); - } - - //@Test - public void testComplexAllocation() throws Throwable { - getRole0Status().setDesired(1); - getRole1Status().setDesired(3); - - List ops = appState.reviewRequestAndReleaseNodes(); - List allocations = engine.execute(ops); - List assignments = new ArrayList<>(); - List releases = new ArrayList<>(); - appState.onContainersAllocated(allocations, assignments, releases); - // we expect four release requests here for all the allocated containers - assertListLength(releases, 4); - for (AbstractRMOperation op : releases) { - assertTrue(op instanceof CancelSingleRequest); - } - assertListLength(assignments, 4); - for (ContainerAssignment assigned : assignments) { - Container target = assigned.container; - RoleInstance ri = roleInstance(assigned); - appState.containerStartSubmitted(target, ri); - } - //insert some async operation here - for (ContainerAssignment assigned : assignments) { - Container target = assigned.container; - appState.innerOnNodeManagerContainerStarted(target.getId()); - } - assertEquals(4, engine.containerCount()); - getRole1Status().setDesired(0); - ops = appState.reviewRequestAndReleaseNodes(); - assertListLength(ops, 3); - allocations = engine.execute(ops); - assertEquals(1, engine.containerCount()); - - appState.onContainersAllocated(allocations, assignments, releases); - assertTrue(assignments.isEmpty()); - assertTrue(releases.isEmpty()); - } - - //@Test - public void testDoubleNodeManagerStartEvent() throws Throwable { - getRole0Status().setDesired(1); - - List ops = appState.reviewRequestAndReleaseNodes(); - List allocations = engine.execute(ops); - List assignments = new ArrayList<>(); - List releases = new ArrayList<>(); - appState.onContainersAllocated(allocations, assignments, releases); - assertListLength(assignments, 1); - ContainerAssignment assigned = assignments.get(0); - Container target = assigned.container; - RoleInstance ri = roleInstance(assigned); - appState.containerStartSubmitted(target, ri); - RoleInstance ri2 = appState.innerOnNodeManagerContainerStarted(target - .getId()); - assertEquals(ri2, ri); - //try a second time, expect an error - try { - appState.innerOnNodeManagerContainerStarted(target.getId()); - fail("Expected an exception"); - } catch (RuntimeException expected) { - // expected - } - //and non-faulter should not downgrade to a null - LOG.warn("Ignore any exception/stack trace that appears below"); - LOG.warn("==============================================================="); - RoleInstance ri3 = appState.onNodeManagerContainerStarted(target.getId()); - LOG.warn("==============================================================="); - LOG.warn("Ignore any exception/stack trace that appeared above"); - assertNull(ri3); - } - - //@Test - public void testDoubleAllocate() throws Throwable { - getRole0Status().setDesired(1); - - List ops = appState.reviewRequestAndReleaseNodes(); - ContainerRequestOperation operation = (ContainerRequestOperation)ops.get(0); - AMRMClient.ContainerRequest request = operation.getRequest(); - Container cont = engine.allocateContainer(request); - List allocated = new ArrayList<>(); - allocated.add(cont); - List assignments = new ArrayList<>(); - List operations = new ArrayList<>(); - assertEquals(0L, getRole0Status().getRunning()); - assertEquals(1L, getRole0Status().getRequested()); - appState.onContainersAllocated(allocated, assignments, operations); - - assertListLength(ops, 1); - assertListLength(assignments, 1); - ContainerAssignment assigned = assignments.get(0); - Container target = assigned.container; - assertEquals(target.getId(), cont.getId()); - int roleId = assigned.role.getPriority(); - assertEquals(roleId, extractRole(request.getPriority())); - assertEquals(assigned.role.getName(), ROLE0); - RoleInstance ri = roleInstance(assigned); - //tell the app it arrived - appState.containerStartSubmitted(target, ri); - appState.innerOnNodeManagerContainerStarted(target.getId()); - assertEquals(1L, getRole0Status().getRunning()); - assertEquals(0L, getRole0Status().getRequested()); - - // now get an extra allocation that should be released - cont = engine.allocateContainer(request); - allocated = new ArrayList<>(); - allocated.add(cont); - assignments = new ArrayList<>(); - operations = new ArrayList<>(); - appState.onContainersAllocated(allocated, assignments, operations); - - assertListLength(operations, 1); - assertTrue(operations.get(0) instanceof ContainerReleaseOperation); - ContainerReleaseOperation release = (ContainerReleaseOperation) - operations.get(0); - assertEquals(release.getContainerId(), cont.getId()); - - assertEquals(1L, getRole0Status().getRunning()); - assertEquals(0L, getRole0Status().getRequested()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRebuildOnAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRebuildOnAMRestart.java deleted file mode 100644 index d257248..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRebuildOnAMRestart.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.slider.api.resource.Application; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockAppState; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.state.AppStateBindingInfo; -import org.apache.slider.server.appmaster.state.NodeEntry; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.NodeMap; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -/** - * Test that app state is rebuilt on a restart. - */ -public class TestMockAppStateRebuildOnAMRestart extends BaseMockAppStateTest - implements MockRoles { - - @Override - public String getTestName() { - return "TestMockAppStateRebuildOnAMRestart"; - } - - //@Test - public void testRebuild() throws Throwable { - - int r0 = 1; - int r1 = 2; - int r2 = 3; - getRole0Status().setDesired(r0); - getRole1Status().setDesired(r1); - getRole2Status().setDesired(r2); - List instances = createAndStartNodes(); - - int clusterSize = r0 + r1 + r2; - assertEquals(instances.size(), clusterSize); - - //clone the list - List containers = new ArrayList<>(); - for (RoleInstance ri : instances) { - containers.add(ri.container); - } - NodeMap nodemap = appState.getRoleHistory().cloneNodemap(); - - //and rebuild - - AppStateBindingInfo bindingInfo = buildBindingInfo(); - bindingInfo.application = factory.newApplication(r0, r1, r2) - .name(getValidTestName()); - bindingInfo.liveContainers = containers; - appState = new MockAppState(bindingInfo); - - assertEquals(appState.getLiveContainers().size(), clusterSize); - - appState.getRoleHistory().dump(); - - //check that the app state direct structures match - List r0live = appState.enumLiveNodesInRole(ROLE0); - List r1live = appState.enumLiveNodesInRole(ROLE1); - List r2live = appState.enumLiveNodesInRole(ROLE2); - - assertEquals(r0, r0live.size()); - assertEquals(r1, r1live.size()); - assertEquals(r2, r2live.size()); - - //now examine the role history - NodeMap newNodemap = appState.getRoleHistory().cloneNodemap(); - - for (NodeInstance nodeInstance : newNodemap.values()) { - String hostname = nodeInstance.hostname; - NodeInstance orig = nodemap.get(hostname); - assertNotNull("Null entry in original nodemap for " + hostname, orig); - - for (int i : Arrays.asList(getRole0Status().getKey(), getRole1Status() - .getKey(), getRole2Status().getKey())) { - assertEquals(nodeInstance.getActiveRoleInstances(i), orig - .getActiveRoleInstances(i)); - NodeEntry origRE = orig.getOrCreate(i); - NodeEntry newRE = nodeInstance.getOrCreate(i); - assertEquals(origRE.getLive(), newRE.getLive()); - assertEquals(0, newRE.getStarting()); - } - } - assertEquals(0, appState.reviewRequestAndReleaseNodes().size()); - - Application application = appState.getClusterStatus(); - // verify the AM restart container count was set - Long restarted = application.getNumberOfRunningContainers(); - assertNotNull(restarted); - //and that the count == 1 master + the region servers - assertEquals(restarted.longValue(), (long)containers.size()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRolePlacement.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRolePlacement.java deleted file mode 100644 index 2eccd1b..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRolePlacement.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.CancelSingleRequest; -import org.apache.slider.server.appmaster.operations.ContainerReleaseOperation; -import org.apache.slider.server.appmaster.operations.ContainerRequestOperation; -import org.apache.slider.server.appmaster.state.ContainerAssignment; -import org.apache.slider.server.appmaster.state.RoleHistoryUtils; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -import static org.apache.slider.server.appmaster.state.ContainerPriority.extractRole; - -/** - * Test that the app state lets you ask for nodes, get a specific host, - * release it and then get that one back again. - */ -public class TestMockAppStateRolePlacement extends BaseMockAppStateTest - implements MockRoles { - - @Override - public String getTestName() { - return "TestMockAppStateRolePlacement"; - } - - - //@Test - public void testAllocateReleaseRealloc() throws Throwable { - getRole0Status().setDesired(1); - - List ops = appState.reviewRequestAndReleaseNodes(); - ContainerRequestOperation operation = (ContainerRequestOperation)ops - .get(0); - AMRMClient.ContainerRequest request = operation.getRequest(); - assertTrue(request.getRelaxLocality()); - assertNull(request.getNodes()); - assertNull(request.getRacks()); - assertNotNull(request.getCapability()); - - Container allocated = engine.allocateContainer(request); - List assignments = new ArrayList<>(); - List releaseOperations = new ArrayList<>(); - appState.onContainersAllocated(Arrays.asList((Container)allocated), - assignments, releaseOperations); - // verify the release matches the allocation - assertEquals(releaseOperations.size(), 1); - CancelSingleRequest cancelOp = (CancelSingleRequest)releaseOperations - .get(0); - assertNotNull(cancelOp.getRequest()); - assertNotNull(cancelOp.getRequest().getCapability()); - assertEquals(cancelOp.getRequest().getCapability(), allocated - .getResource()); - // now the assignment - assertEquals(assignments.size(), 1); - ContainerAssignment assigned = assignments.get(0); - Container container = assigned.container; - assertEquals(container.getId(), allocated.getId()); - int roleId = assigned.role.getPriority(); - assertEquals(roleId, extractRole(request.getPriority())); - assertEquals(assigned.role.getName(), ROLE0); - String containerHostname = RoleHistoryUtils.hostnameOf(container); - RoleInstance ri = roleInstance(assigned); - //tell the app it arrived - appState.containerStartSubmitted(container, ri); - assertNotNull(appState.onNodeManagerContainerStarted(container.getId())); - assertEquals(getRole0Status().getRunning(), 1); - ops = appState.reviewRequestAndReleaseNodes(); - assertEquals(ops.size(), 0); - - //now it is surplus - getRole0Status().setDesired(0); - ops = appState.reviewRequestAndReleaseNodes(); - ContainerReleaseOperation release = (ContainerReleaseOperation) ops.get(0); - - assertEquals(release.getContainerId(), container.getId()); - engine.execute(ops); - assertNotNull(appState.onCompletedContainer(containerStatus(container - .getId())).roleInstance); - - //view the world - appState.getRoleHistory().dump(); - - //now ask for a new one - getRole0Status().setDesired(1); - ops = appState.reviewRequestAndReleaseNodes(); - assertEquals(ops.size(), 1); - operation = (ContainerRequestOperation) ops.get(0); - AMRMClient.ContainerRequest request2 = operation.getRequest(); - assertNotNull(request2); - assertEquals(request2.getNodes().get(0), containerHostname); - assertFalse(request2.getRelaxLocality()); - engine.execute(ops); - - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRoleRelease.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRoleRelease.java deleted file mode 100644 index b6c8526..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateRoleRelease.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.model.mock.MockYarnEngine; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; - -/** - * Test that if you have >1 role, the right roles are chosen for release. - */ -public class TestMockAppStateRoleRelease extends BaseMockAppStateTest - implements MockRoles { - - @Override - public String getTestName() { - return "TestMockAppStateRoleRelease"; - } - - /** - * Small cluster with multiple containers per node, - * to guarantee many container allocations on each node. - * @return - */ - @Override - public MockYarnEngine createYarnEngine() { - return new MockYarnEngine(4, 4); - } - - //@Test - public void testAllocateReleaseRealloc() throws Throwable { - /** - * Allocate to all nodes - */ - getRole0Status().setDesired(6); - getRole1Status().setDesired(5); - getRole2Status().setDesired(4); - List instances = createAndStartNodes(); - assertEquals(instances.size(), 15); - - //now it is surplus - getRole0Status().setDesired(0); - List ops = appState.reviewRequestAndReleaseNodes(); - - List released = new ArrayList<>(); - engine.execute(ops, released); - List ids = extractContainerIds(instances, ROLE0); - for (ContainerId cid : released) { - assertNotNull(appState.onCompletedContainer(containerStatus(cid)) - .roleInstance); - assertTrue(ids.contains(cid)); - } - - //view the world - appState.getRoleHistory().dump(); - - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateUniqueNames.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateUniqueNames.java deleted file mode 100644 index 8dae8e7..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockAppStateUniqueNames.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.api.resource.Resource; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.model.mock.MockYarnEngine; -import org.apache.slider.server.appmaster.state.AppStateBindingInfo; -import org.apache.slider.server.appmaster.state.MostRecentContainerReleaseSelector; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.TreeMap; - -/** - * Test that if you have more than one role, the right roles are chosen for - * release. - */ -public class TestMockAppStateUniqueNames extends BaseMockAppStateTest - implements MockRoles { - - @Override - public String getTestName() { - return "TestMockAppStateUniqueNames"; - } - - /** - * Small cluster with multiple containers per node, - * to guarantee many container allocations on each node. - * @return - */ - @Override - public MockYarnEngine createYarnEngine() { - return new MockYarnEngine(4, 4); - } - - @Override - public AppStateBindingInfo buildBindingInfo() throws IOException { - AppStateBindingInfo bindingInfo = super.buildBindingInfo(); - bindingInfo.releaseSelector = new MostRecentContainerReleaseSelector(); - return bindingInfo; - } - - @Override - public Application buildApplication() { - Application application = super.buildApplication(); - - Component component = new Component().name("group1").numberOfContainers(2L) - .resource(new Resource().memory("1024").cpus(2)) - .uniqueComponentSupport(true); - - application.getComponents().add(component); - return application; - } - - public static Map organize(List - instances) { - Map map = new TreeMap<>(); - for (RoleInstance instance : instances) { - assertFalse("Multiple role instances for unique name " + instance - .compInstanceName, map.containsKey(instance.compInstanceName)); - System.out.println("Adding to map " + instance.compInstanceName + " for" + - instance.role); - map.put(instance.compInstanceName, instance); - } - return map; - } - - public static void verifyInstances(List instances, String - group, String... roles) { - assertEquals(roles.length, instances.size()); - Map map = organize(instances); - int i = 0; - for (Entry entry : map.entrySet()) { - assertEquals(roles[i], entry.getKey()); - RoleInstance instance = entry.getValue(); - assertEquals(roles[i], instance.compInstanceName); - assertEquals(i, instance.componentId); - assertEquals(group, instance.role); - assertEquals(group, instance.providerRole.name); - i++; - } - } - - //@Test - public void testDynamicFlexDown() throws Throwable { - createAndStartNodes(); - List instances = appState.cloneOwnedContainerList(); - verifyInstances(instances, "group1", "group10", "group11"); - - appState.updateComponents(Collections.singletonMap("group1", 0L)); - createAndStartNodes(); - instances = appState.cloneOwnedContainerList(); - assertEquals(0, instances.size()); - - RoleStatus roleStatus = appState.lookupRoleStatus("group1"); - assertEquals(0, roleStatus.getDesired()); - assertEquals(1024L, roleStatus.getResourceRequirements().getMemorySize()); - assertEquals(2, roleStatus.getResourceRequirements().getVirtualCores()); - - // now flex back up - appState.updateComponents(Collections.singletonMap("group1", 3L)); - createAndStartNodes(); - instances = appState.cloneOwnedContainerList(); - verifyInstances(instances, "group1", "group10", "group11", "group12"); - } - - //@Test - public void testDynamicFlexUp() throws Throwable { - createAndStartNodes(); - List instances = appState.cloneOwnedContainerList(); - verifyInstances(instances, "group1", "group10", "group11"); - - appState.updateComponents(Collections.singletonMap("group1", 3L)); - createAndStartNodes(); - instances = appState.cloneOwnedContainerList(); - verifyInstances(instances, "group1", "group10", "group11", "group12"); - - RoleStatus group1 = appState.lookupRoleStatus("group1"); - assertEquals(3, group1.getDesired()); - assertEquals(1024L, group1.getResourceRequirements().getMemorySize()); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.java deleted file mode 100644 index 8bf2742..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockContainerResourceAllocations.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockAppState; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.ContainerRequestOperation; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; - -import java.util.Collections; -import java.util.List; - -/** - * Test the container resource allocation logic. - */ -public class TestMockContainerResourceAllocations extends BaseMockAppStateTest { - - @Override - public Application buildApplication() { - return factory.newApplication(1, 0, 0).name(getValidTestName()); - } - - //@Test - public void testNormalAllocations() throws Throwable { - Component role0 = appState.getClusterStatus().getComponent(MockRoles.ROLE0); - role0.resource(new org.apache.slider.api.resource.Resource().memory("512") - .cpus(2)); - // hack - because role0 is created before the test run - RoleStatus role0Status = - appState.getRoleStatusMap().get(appState.getRoleMap().get(ROLE0).id); - role0Status.setResourceRequirements( - appState.buildResourceRequirements(role0Status)); - appState.updateComponents(Collections.singletonMap(role0.getName(), - role0.getNumberOfContainers())); - List ops = appState.reviewRequestAndReleaseNodes(); - assertEquals(1, ops.size()); - ContainerRequestOperation operation = (ContainerRequestOperation) ops - .get(0); - Resource requirements = operation.getRequest().getCapability(); - assertEquals(512L, requirements.getMemorySize()); - assertEquals(2, requirements.getVirtualCores()); - } - - //TODO replace with resource profile feature in yarn - //@Test - public void testMaxMemAllocations() throws Throwable { - // max core allocations no longer supported - Component role0 = appState.getClusterStatus().getComponent(MockRoles.ROLE0); - role0.resource(new org.apache.slider.api.resource.Resource() - .memory(ResourceKeys.YARN_RESOURCE_MAX).cpus(2)); - RoleStatus role0Status = - appState.getRoleStatusMap().get(appState.getRoleMap().get(ROLE0).id); - role0Status.setResourceRequirements( - appState.buildResourceRequirements(role0Status)); - appState.updateComponents(Collections.singletonMap(role0.getName(), - role0.getNumberOfContainers())); - List ops = appState.reviewRequestAndReleaseNodes(); - assertEquals(1, ops.size()); - ContainerRequestOperation operation = (ContainerRequestOperation) ops - .get(0); - Resource requirements = operation.getRequest().getCapability(); - assertEquals(MockAppState.RM_MAX_RAM, requirements.getMemorySize()); - assertEquals(2, requirements.getVirtualCores()); - } - - //@Test - public void testMaxDefaultAllocations() throws Throwable { - List ops = appState.reviewRequestAndReleaseNodes(); - assertEquals(ops.size(), 1); - ContainerRequestOperation operation = (ContainerRequestOperation) ops - .get(0); - Resource requirements = operation.getRequest().getCapability(); - assertEquals(ResourceKeys.DEF_YARN_MEMORY, requirements.getMemorySize()); - assertEquals(ResourceKeys.DEF_YARN_CORES, requirements.getVirtualCores()); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockLabelledAAPlacement.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockLabelledAAPlacement.java deleted file mode 100644 index 453b14c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestMockLabelledAAPlacement.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.NodeState; -import org.apache.slider.server.appmaster.model.mock.MockFactory; -import org.apache.slider.server.appmaster.model.mock.MockRoles; -import org.apache.slider.server.appmaster.model.mock.MockYarnEngine; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.AppState.NodeUpdatedOutcome; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; - -/** - * Test Anti-affine placement. - */ -public class TestMockLabelledAAPlacement extends BaseMockAppStateAATest - implements MockRoles { - private static final Logger LOG = - LoggerFactory.getLogger(TestMockLabelledAAPlacement.class); - - private static final int NODES = 3; - private static final int GPU_NODES = 2; - private static final String HOST0 = "00000000"; - private static final String HOST1 = "00000001"; - - @Override - public void setup() throws Exception { - super.setup(); - - updateNodes(MockFactory.INSTANCE.newNodeReport(HOST0, NodeState.RUNNING, - LABEL_GPU)); - updateNodes(MockFactory.INSTANCE.newNodeReport(HOST1, NodeState.RUNNING, - LABEL_GPU)); - } - - @Override - public MockYarnEngine createYarnEngine() { - return new MockYarnEngine(NODES, 8); - } - - void assertAllContainersAA() { - assertAllContainersAA(getGpuRole().getKey()); - } - - /** - * - * @throws Throwable - */ - //@Test - public void testAskForTooMany() throws Throwable { - RoleStatus gpuRole = getGpuRole(); - - describe("Ask for 1 more than the no of available nodes;" + - " expect the final request to be unsatisfied until the cluster " + - "changes size"); - //more than expected - int size = GPU_NODES; - gpuRole.setDesired(size + 1); - - List operations = appState - .reviewRequestAndReleaseNodes(); - assertTrue(gpuRole.isAARequestOutstanding()); - - assertEquals(gpuRole.getAAPending(), size); - for (int i = 0; i < size; i++) { - String iter = "Iteration " + i + " role = " + getAaRole(); - describe(iter); - List operationsOut = new ArrayList<>(); - - List roleInstances = submitOperations(operations, - EMPTY_ID_LIST, operationsOut); - // one instance per request - assertEquals(1, roleInstances.size()); - appState.onNodeManagerContainerStarted(roleInstances.get(0) - .getContainerId()); - assertAllContainersAA(); - // there should be none left - LOG.debug(nodeInformationSnapshotAsString()); - operations = operationsOut; - if (i + 1 < size) { - assertEquals(2, operations.size()); - } else { - assertEquals(1, operations.size()); - } - } - // expect an outstanding AA request to be unsatisfied - assertTrue(gpuRole.getRunning() < gpuRole.getDesired()); - assertEquals(0, gpuRole.getRequested()); - assertFalse(gpuRole.isAARequestOutstanding()); - List allocatedContainers = engine.execute(operations, - EMPTY_ID_LIST); - assertEquals(0, allocatedContainers.size()); - // in a review now, no more requests can be generated, as there is no - // space for AA placements, even though there is cluster capacity - assertEquals(0, appState.reviewRequestAndReleaseNodes().size()); - - // switch node 2 into being labelled - NodeUpdatedOutcome outcome = updateNodes(MockFactory.INSTANCE. - newNodeReport("00000002", NodeState.RUNNING, "gpu")); - - assertEquals(NODES, cloneNodemap().size()); - assertTrue(outcome.clusterChanged); - // no active calls to empty - assertTrue(outcome.operations.isEmpty()); - assertEquals(1, appState.reviewRequestAndReleaseNodes().size()); - } - - protected AppState.NodeUpdatedOutcome addNewNode() { - return updateNodes(MockFactory.INSTANCE.newNodeReport("00000004", - NodeState.RUNNING, "gpu")); - } - - //@Test - public void testClusterSizeChangesDuringRequestSequence() throws Throwable { - RoleStatus gpuRole = getGpuRole(); - describe("Change the cluster size where the cluster size changes during " + - "a test sequence."); - gpuRole.setDesired(GPU_NODES + 1); - List operations = appState - .reviewRequestAndReleaseNodes(); - assertTrue(gpuRole.isAARequestOutstanding()); - assertEquals(GPU_NODES, gpuRole.getAAPending()); - NodeUpdatedOutcome outcome = addNewNode(); - assertTrue(outcome.clusterChanged); - // one call to cancel - assertEquals(1, outcome.operations.size()); - // and on a review, one more to rebuild - assertEquals(1, appState.reviewRequestAndReleaseNodes().size()); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestOutstandingRequestValidation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestOutstandingRequestValidation.java deleted file mode 100644 index 65d1d86..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/appstate/TestOutstandingRequestValidation.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * 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.slider.server.appmaster.model.appstate; - -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; -import org.apache.slider.server.appmaster.state.ContainerPriority; -import org.apache.slider.server.appmaster.state.OutstandingRequest; -import org.apache.slider.utils.SliderTestBase; -import org.junit.Test; - -/** - * Test outstanding request validation. - */ -public class TestOutstandingRequestValidation extends SliderTestBase { - - private static final String[] H1 = hosts("one"); - - //@Test - public void testRelaxedNohostsOrLabels() throws Throwable { - createAndValidate(null, null, true); - } - - //@Test - public void testRelaxedLabels() throws Throwable { - createAndValidate(null, "gpu", true); - } - - //@Test - public void testNonRelaxedLabels() throws Throwable { - expectCreationFailure(null, "gpu", false); - } - - //@Test - public void testRelaxedHostNoLabel() throws Throwable { - createAndValidate(H1, "", true); - } - - /** - * Use varargs for simple list to array conversion. - * @param hostnames host names - * @return - */ - public static String[] hosts(String...hostnames) { - return hostnames; - } - - void expectCreationFailure( - String[] hosts, - String labels, - boolean relaxLocality) { - try { - ContainerRequest result = createAndValidate(hosts, labels, relaxLocality); - fail("Expected an exception, got " + result); - } catch (IllegalArgumentException expected) { - assertTrue(expected.toString() - .contains("Can't turn off locality relaxation on a request with no " + - "location constraints")); - } - } - - AMRMClient.ContainerRequest createAndValidate( - String[] hosts, - String labels, - boolean relaxLocality) { - int cores = 1; - int memory = 64; - int p = 1; - Priority pri = ContainerPriority.createPriority(p, !relaxLocality); - ContainerRequest issuedRequest = - newRequest(pri, hosts, labels, relaxLocality); - OutstandingRequest.validateContainerRequest(issuedRequest, p, ""); - return issuedRequest; - } - - AMRMClient.ContainerRequest newRequest( - Priority pri, - String[] hosts, - String labels, - boolean relaxLocality) { - int cores = 1; - int memory = 64; - Resource resource = Resource.newInstance(memory, cores); - return new AMRMClient.ContainerRequest(resource, - hosts, - null, - pri, - relaxLocality, - labels); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryAA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryAA.java deleted file mode 100644 index 0cdb952..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryAA.java +++ /dev/null @@ -1,269 +0,0 @@ -/* - * 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.slider.server.appmaster.model.history; - -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.NodeState; -import org.apache.slider.api.proto.Messages; -import org.apache.slider.api.types.NodeInformation; -import org.apache.slider.api.types.NodeInformationList; -import org.apache.slider.api.types.RestTypeMarshalling; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.server.appmaster.model.mock.MockFactory; -import org.apache.slider.server.appmaster.model.mock.MockRoleHistory; -import org.apache.slider.server.appmaster.state.NodeEntry; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.NodeMap; -import org.apache.slider.server.appmaster.state.RoleHistory; -import org.apache.slider.utils.SliderTestBase; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -/** - * Test anti-affine placement. - */ -public class TestRoleHistoryAA extends SliderTestBase { - private static final Logger LOG = - LoggerFactory.getLogger(TestRoleHistoryAA.class); - - private List hostnames = Arrays.asList("1", "2", "3"); - private NodeMap nodeMap, gpuNodeMap; - private RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - - public TestRoleHistoryAA() throws BadConfigException { - } - - @Override - public void setup() throws Exception { - super.setup(); - nodeMap = createNodeMap(hostnames, NodeState.RUNNING, ""); - gpuNodeMap = createNodeMap(hostnames, NodeState.RUNNING, "GPU"); - } - - //@Test - public void testFindNodesInFullCluster() throws Throwable { - // all three will surface at first - verifyResultSize(3, nodeMap.findAllNodesForRole(1, "")); - } - - //@Test - public void testFindNodesInUnhealthyCluster() throws Throwable { - // all three will surface at first - markNodeOneUnhealthy(); - verifyResultSize(2, nodeMap.findAllNodesForRole(1, "")); - } - - public boolean markNodeOneUnhealthy() { - return setNodeState(nodeMap.get("1"), NodeState.UNHEALTHY); - } - - protected boolean setNodeState(NodeInstance node, NodeState state) { - return node.updateNode(MockFactory.INSTANCE.newNodeReport(node.hostname, - state, "")); - } - - //@Test - public void testFindNoNodesWrongLabel() throws Throwable { - // all three will surface at first - verifyResultSize(0, nodeMap.findAllNodesForRole(1, "GPU")); - } - - //@Test - public void testFindSomeNodesSomeLabel() throws Throwable { - // all three will surface at first - update(nodeMap, - Arrays.asList(MockFactory.INSTANCE.newNodeReport("1", NodeState - .RUNNING, "GPU"))); - List gpuNodes = nodeMap.findAllNodesForRole(1, "GPU"); - verifyResultSize(1, gpuNodes); - NodeInstance instance = gpuNodes.get(0); - instance.getOrCreate(1).onStarting(); - assertFalse(instance.canHost(1, "GPU")); - assertFalse(instance.canHost(1, "")); - verifyResultSize(0, nodeMap.findAllNodesForRole(1, "GPU")); - - } - - //@Test - public void testFindNoNodesRightLabel() throws Throwable { - // all three will surface at first - verifyResultSize(3, gpuNodeMap.findAllNodesForRole(1, "GPU")); - } - - //@Test - public void testFindNoNodesNoLabel() throws Throwable { - // all three will surface at first - verifyResultSize(3, gpuNodeMap.findAllNodesForRole(1, "")); - } - - //@Test - public void testFindNoNodesClusterRequested() throws Throwable { - // all three will surface at first - for (NodeInstance ni : nodeMap.values()) { - ni.getOrCreate(1).request(); - } - assertNoAvailableNodes(1); - } - - //@Test - public void testFindNoNodesClusterBusy() throws Throwable { - // all three will surface at first - for (NodeInstance ni : nodeMap.values()) { - ni.getOrCreate(1).request(); - } - assertNoAvailableNodes(1); - } - - /** - * Tag all nodes as starting, then walk one through a bit - * more of its lifecycle. - */ - //@Test - public void testFindNoNodesLifecycle() throws Throwable { - // all three will surface at first - for (NodeInstance ni : nodeMap.values()) { - ni.getOrCreate(1).onStarting(); - } - assertNoAvailableNodes(1); - - // walk one of the nodes through the lifecycle - NodeInstance node1 = nodeMap.get("1"); - assertFalse(node1.canHost(1, "")); - node1.get(1).onStartCompleted(); - assertFalse(node1.canHost(1, "")); - assertNoAvailableNodes(1); - node1.get(1).release(); - assertTrue(node1.canHost(1, "")); - List list2 = - verifyResultSize(1, nodeMap.findAllNodesForRole(1, "")); - assertEquals(list2.get(0).hostname, "1"); - - // now tag that node as unhealthy and expect it to go away - markNodeOneUnhealthy(); - assertNoAvailableNodes(1); - } - - //@Test - public void testRolesIndependent() throws Throwable { - NodeInstance node1 = nodeMap.get("1"); - NodeEntry role1 = node1.getOrCreate(1); - NodeEntry role2 = node1.getOrCreate(2); - for (NodeInstance ni : nodeMap.values()) { - ni.updateNode(MockFactory.INSTANCE.newNodeReport("0", NodeState - .UNHEALTHY, "")); - } - assertNoAvailableNodes(1); - assertNoAvailableNodes(2); - assertTrue(setNodeState(node1, NodeState.RUNNING)); - // tag role 1 as busy - role1.onStarting(); - assertNoAvailableNodes(1); - - verifyResultSize(1, nodeMap.findAllNodesForRole(2, "")); - assertTrue(node1.canHost(2, "")); - } - - //@Test - public void testNodeEntryAvailablity() throws Throwable { - NodeEntry entry = new NodeEntry(1); - assertTrue(entry.isAvailable()); - entry.onStarting(); - assertFalse(entry.isAvailable()); - entry.onStartCompleted(); - assertFalse(entry.isAvailable()); - entry.release(); - assertTrue(entry.isAvailable()); - entry.onStarting(); - assertFalse(entry.isAvailable()); - entry.onStartFailed(); - assertTrue(entry.isAvailable()); - } - - //@Test - public void testNodeInstanceSerialization() throws Throwable { - MockRoleHistory rh2 = new MockRoleHistory(new ArrayList<>()); - rh2.getOrCreateNodeInstance("localhost"); - NodeInstance instance = rh2.getOrCreateNodeInstance("localhost"); - instance.getOrCreate(1).onStartCompleted(); - Map naming = Collections.singletonMap(1, "manager"); - NodeInformation ni = instance.serialize(naming); - assertEquals(1, ni.entries.get("manager").live); - NodeInformation ni2 = rh2.getNodeInformation("localhost", naming); - assertEquals(1, ni2.entries.get("manager").live); - Map info = rh2.getNodeInformationSnapshot(naming); - assertEquals(1, info.get("localhost").entries.get("manager").live); - NodeInformationList nil = new NodeInformationList(info.values()); - assertEquals(1, nil.get(0).entries.get("manager").live); - - Messages.NodeInformationProto nodeInformationProto = - RestTypeMarshalling.marshall(ni); - Messages.NodeEntryInformationProto entryProto = nodeInformationProto - .getEntries(0); - assertNotNull(entryProto); - assertEquals(1, entryProto.getPriority()); - NodeInformation unmarshalled = - RestTypeMarshalling.unmarshall(nodeInformationProto); - assertEquals(unmarshalled.hostname, ni.hostname); - assertTrue(unmarshalled.entries.keySet().containsAll(ni.entries.keySet())); - - } - - //@Test - public void testBuildRolenames() throws Throwable { - - } - public List assertNoAvailableNodes(int role) { - String label = ""; - return verifyResultSize(0, nodeMap.findAllNodesForRole(role, label)); - } - - List verifyResultSize(int size, List list) { - if (list.size() != size) { - for (NodeInstance ni : list) { - LOG.error(ni.toFullString()); - } - } - assertEquals(size, list.size()); - return list; - } - - NodeMap createNodeMap(List nodeReports) - throws BadConfigException { - NodeMap newNodeMap = new NodeMap(1); - update(newNodeMap, nodeReports); - return newNodeMap; - } - - protected boolean update(NodeMap nm, List nodeReports) { - return nm.buildOrUpdate(nodeReports); - } - - NodeMap createNodeMap(List hosts, NodeState state, - String label) throws BadConfigException { - return createNodeMap(MockFactory.INSTANCE.createNodeReports(hosts, state, - label)); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.java deleted file mode 100644 index bbe95b9..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryContainerEvents.java +++ /dev/null @@ -1,447 +0,0 @@ -/* - * 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.slider.server.appmaster.model.history; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.NodeState; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockContainer; -import org.apache.slider.server.appmaster.model.mock.MockNodeId; -import org.apache.slider.server.appmaster.state.ContainerOutcome; -import org.apache.slider.server.appmaster.state.ContainerPriority; -import org.apache.slider.server.appmaster.state.NodeEntry; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.NodeMap; -import org.apache.slider.server.appmaster.state.RoleHistory; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -/** - * Test container events at the role history level -one below - * the App State. - */ -public class TestRoleHistoryContainerEvents extends BaseMockAppStateTest { - private static final Logger LOG = - LoggerFactory.getLogger(TestRoleHistoryContainerEvents.class); - - @Override - public String getTestName() { - return "TestRoleHistoryContainerEvents"; - } - - private NodeInstance age1Active4; - private NodeInstance age2Active2; - private NodeInstance age3Active0; - private NodeInstance age4Active1; - private NodeInstance age2Active0; - - private RoleHistory roleHistory; - - private Resource resource; - - AMRMClient.ContainerRequest requestContainer(RoleStatus roleStatus) { - return roleHistory.requestContainerForRole(roleStatus).getIssuedRequest(); - } - - @Override - public void setup() throws Exception { - super.setup(); - - age1Active4 = nodeInstance(1, 4, 0, 0); - age2Active2 = nodeInstance(2, 2, 0, 1); - age3Active0 = nodeInstance(3, 0, 0, 0); - age4Active1 = nodeInstance(4, 1, 0, 0); - age2Active0 = nodeInstance(2, 0, 0, 0); - - roleHistory = appState.getRoleHistory(); - roleHistory.insert(Arrays.asList(age2Active2, age2Active0, - age4Active1, age1Active4, age3Active0)); - roleHistory.buildRecentNodeLists(); - resource = Resource.newInstance(ResourceKeys.DEF_YARN_CORES, - ResourceKeys.DEF_YARN_MEMORY); - } - - //@Test - public void testFindAndCreate() throws Throwable { - RoleStatus roleStatus = getRole0Status(); - - AMRMClient.ContainerRequest request = - requestContainer(roleStatus); - - List requestNodes = request.getNodes(); - assertNotNull(requestNodes); - assertEquals(1, requestNodes.size()); - String hostname = requestNodes.get(0); - assertEquals(hostname, age3Active0.hostname); - - //build a container - MockContainer container = factory.newContainer(); - container.setNodeId(new MockNodeId(hostname, 0)); - container.setPriority(request.getPriority()); - roleHistory.onContainerAssigned(container); - - NodeMap nodemap = roleHistory.cloneNodemap(); - NodeInstance allocated = nodemap.get(hostname); - NodeEntry roleEntry = allocated.get(roleStatus.getKey()); - assertEquals(1, roleEntry.getStarting()); - assertFalse(roleEntry.isAvailable()); - RoleInstance ri = new RoleInstance(container); - //start it - roleHistory.onContainerStartSubmitted(container, ri); - //later, declare that it started - roleHistory.onContainerStarted(container); - assertEquals(0, roleEntry.getStarting()); - assertFalse(roleEntry.isAvailable()); - assertEquals(1, roleEntry.getActive()); - assertEquals(1, roleEntry.getLive()); - } - - //@Test - public void testCreateAndRelease() throws Throwable { - RoleStatus roleStatus = getRole1Status(); - - //verify it is empty - assertTrue(roleHistory.listActiveNodes(roleStatus.getKey()).isEmpty()); - - AMRMClient.ContainerRequest request = - requestContainer(roleStatus); - - assertNull(request.getNodes()); - - //pick an idle host - String hostname = age3Active0.hostname; - - //build a container - MockContainer container = factory.newContainer(new MockNodeId(hostname, - 0), request.getPriority()); - roleHistory.onContainerAssigned(container); - - NodeMap nodemap = roleHistory.cloneNodemap(); - NodeInstance allocated = nodemap.get(hostname); - NodeEntry roleEntry = allocated.get(roleStatus.getKey()); - assertEquals(1, roleEntry.getStarting()); - assertFalse(roleEntry.isAvailable()); - RoleInstance ri = new RoleInstance(container); - //start it - roleHistory.onContainerStartSubmitted(container, ri); - //later, declare that it started - roleHistory.onContainerStarted(container); - assertEquals(0, roleEntry.getStarting()); - assertFalse(roleEntry.isAvailable()); - assertEquals(1, roleEntry.getActive()); - assertEquals(1, roleEntry.getLive()); - - // now pick that instance to destroy - List activeNodes = roleHistory.listActiveNodes(roleStatus - .getKey()); - - - assertEquals(1, activeNodes.size()); - NodeInstance target = activeNodes.get(0); - assertEquals(target, allocated); - roleHistory.onContainerReleaseSubmitted(container); - assertEquals(1, roleEntry.getReleasing()); - assertEquals(1, roleEntry.getLive()); - assertEquals(0, roleEntry.getActive()); - - // release completed - roleHistory.onReleaseCompleted(container); - assertEquals(0, roleEntry.getReleasing()); - assertEquals(0, roleEntry.getLive()); - assertEquals(0, roleEntry.getActive()); - - // verify it is empty - assertTrue(roleHistory.listActiveNodes(roleStatus.getKey()).isEmpty()); - - // ask for a container and expect to get the recently released one - AMRMClient.ContainerRequest request2 = - requestContainer(roleStatus); - - List nodes2 = request2.getNodes(); - assertNotNull(nodes2); - String hostname2 = nodes2.get(0); - - //pick an idle host - assertEquals(hostname2, age3Active0.hostname); - } - - - //@Test - public void testStartWithoutWarning() throws Throwable { - //pick an idle host - String hostname = age3Active0.hostname; - //build a container - MockContainer container = factory.newContainer( - new MockNodeId(hostname, 0), - ContainerPriority.createPriority(getRole0Status().getKey(), false)); - - NodeMap nodemap = roleHistory.cloneNodemap(); - NodeInstance allocated = nodemap.get(hostname); - NodeEntry roleEntry = allocated.get(getRole0Status().getKey()); - - //tell RH that it started - roleHistory.onContainerStarted(container); - assertEquals(0, roleEntry.getStarting()); - assertFalse(roleEntry.isAvailable()); - assertEquals(1, roleEntry.getActive()); - assertEquals(1, roleEntry.getLive()); - } - - //@Test - public void testStartFailed() throws Throwable { - RoleStatus roleStatus = getRole0Status(); - - AMRMClient.ContainerRequest request = - requestContainer(roleStatus); - - LOG.info("req {}", request); - LOG.info("{}", request.getNodes()); - String hostname = request.getNodes().get(0); - assertEquals(hostname, age3Active0.hostname); - - //build a container - MockContainer container = factory.newContainer(new MockNodeId(hostname, - 0), request.getPriority()); - roleHistory.onContainerAssigned(container); - - NodeMap nodemap = roleHistory.cloneNodemap(); - NodeInstance allocated = nodemap.get(hostname); - NodeEntry roleEntry = allocated.get(roleStatus.getKey()); - assertEquals(1, roleEntry.getStarting()); - assertFalse(roleEntry.isAvailable()); - RoleInstance ri = new RoleInstance(container); - //start it - roleHistory.onContainerStartSubmitted(container, ri); - //later, declare that it failed on startup - assertFalse(roleHistory.onNodeManagerContainerStartFailed(container)); - assertEquals(0, roleEntry.getStarting()); - assertEquals(1, roleEntry.getStartFailed()); - assertEquals(1, roleEntry.getFailed()); - assertTrue(roleEntry.isAvailable()); - assertEquals(0, roleEntry.getActive()); - assertEquals(0, roleEntry.getLive()); - } - - //@Test - public void testStartFailedWithoutWarning() throws Throwable { - RoleStatus roleStatus = getRole0Status(); - - AMRMClient.ContainerRequest request = - requestContainer(roleStatus); - - String hostname = request.getNodes().get(0); - assertEquals(hostname, age3Active0.hostname); - - //build a container - MockContainer container = factory.newContainer(); - container.setNodeId(new MockNodeId(hostname, 0)); - container.setPriority(request.getPriority()); - - NodeMap nodemap = roleHistory.cloneNodemap(); - NodeInstance allocated = nodemap.get(hostname); - NodeEntry roleEntry = allocated.get(roleStatus.getKey()); - - assertFalse(roleHistory.onNodeManagerContainerStartFailed(container)); - assertEquals(0, roleEntry.getStarting()); - assertEquals(1, roleEntry.getStartFailed()); - assertEquals(1, roleEntry.getFailed()); - assertTrue(roleEntry.isAvailable()); - assertEquals(0, roleEntry.getActive()); - assertEquals(0, roleEntry.getLive()); - } - - //@Test - public void testContainerFailed() throws Throwable { - describe("fail a container without declaring it as starting"); - - RoleStatus roleStatus = getRole0Status(); - - AMRMClient.ContainerRequest request = - requestContainer(roleStatus); - - String hostname = request.getNodes().get(0); - assertEquals(hostname, age3Active0.hostname); - - //build a container - MockContainer container = factory.newContainer(); - container.setNodeId(new MockNodeId(hostname, 0)); - container.setPriority(request.getPriority()); - roleHistory.onContainerAssigned(container); - - NodeMap nodemap = roleHistory.cloneNodemap(); - NodeInstance allocated = nodemap.get(hostname); - NodeEntry roleEntry = allocated.get(roleStatus.getKey()); - assertEquals(1, roleEntry.getStarting()); - assertFalse(roleEntry.isAvailable()); - RoleInstance ri = new RoleInstance(container); - //start it - roleHistory.onContainerStartSubmitted(container, ri); - roleHistory.onContainerStarted(container); - - //later, declare that it failed - roleHistory.onFailedContainer( - container, - false, - ContainerOutcome.Failed); - assertEquals(0, roleEntry.getStarting()); - assertTrue(roleEntry.isAvailable()); - assertEquals(0, roleEntry.getActive()); - assertEquals(0, roleEntry.getLive()); - } - - //@Test - public void testContainerFailedWithoutWarning() throws Throwable { - describe("fail a container without declaring it as starting"); - RoleStatus roleStatus = getRole0Status(); - - AMRMClient.ContainerRequest request = - requestContainer(roleStatus); - - String hostname = request.getNodes().get(0); - assertEquals(hostname, age3Active0.hostname); - - //build a container - MockContainer container = factory.newContainer(); - container.setNodeId(new MockNodeId(hostname, 0)); - container.setPriority(request.getPriority()); - - - NodeMap nodemap = roleHistory.cloneNodemap(); - NodeInstance allocated = nodemap.get(hostname); - NodeEntry roleEntry = allocated.get(roleStatus.getKey()); - assertTrue(roleEntry.isAvailable()); - roleHistory.onFailedContainer( - container, - false, - ContainerOutcome.Failed); - assertEquals(0, roleEntry.getStarting()); - assertEquals(1, roleEntry.getFailed()); - assertTrue(roleEntry.isAvailable()); - assertEquals(0, roleEntry.getActive()); - assertEquals(0, roleEntry.getLive()); - } - - //@Test - public void testAllocationListPrep() throws Throwable { - describe("test prepareAllocationList"); - RoleStatus roleStatus = getRole0Status(); - - AMRMClient.ContainerRequest request = - requestContainer(roleStatus); - - String hostname = request.getNodes().get(0); - assertEquals(hostname, age3Active0.hostname); - - MockContainer container1 = factory.newContainer(); - container1.setNodeId(new MockNodeId(hostname, 0)); - container1.setPriority(Priority.newInstance(getRole0Status().getKey())); - - MockContainer container2 = factory.newContainer(); - container2.setNodeId(new MockNodeId(hostname, 0)); - container2.setPriority(Priority.newInstance(getRole1Status().getKey())); - - // put containers in List with role == 1 first - List containers = Arrays.asList((Container) container2, - (Container) container1); - List sortedContainers = roleHistory.prepareAllocationList( - containers); - - // verify that the first container has role == 0 after sorting - MockContainer c1 = (MockContainer) sortedContainers.get(0); - assertEquals(getRole0Status().getKey(), c1.getPriority().getPriority()); - MockContainer c2 = (MockContainer) sortedContainers.get(1); - assertEquals(getRole1Status().getKey(), c2.getPriority().getPriority()); - } - - //@Test - public void testNodeUpdated() throws Throwable { - describe("fail a node"); - - RoleStatus roleStatus = getRole0Status(); - - AMRMClient.ContainerRequest request = - roleHistory.requestContainerForRole(roleStatus).getIssuedRequest(); - - String hostname = request.getNodes().get(0); - assertEquals(age3Active0.hostname, hostname); - - // build a container - MockContainer container = factory.newContainer(new MockNodeId(hostname, - 0), request.getPriority()); - - roleHistory.onContainerAssigned(container); - - NodeMap nodemap = roleHistory.cloneNodemap(); - NodeInstance allocated = nodemap.get(hostname); - NodeEntry roleEntry = allocated.get(roleStatus.getKey()); - assertEquals(1, roleEntry.getStarting()); - assertFalse(roleEntry.isAvailable()); - RoleInstance ri = new RoleInstance(container); - // start it - roleHistory.onContainerStartSubmitted(container, ri); - roleHistory.onContainerStarted(container); - - int startSize = nodemap.size(); - - // now send a list of updated (failed) nodes event - List nodesUpdated = new ArrayList<>(); - NodeReport nodeReport = NodeReport.newInstance( - NodeId.newInstance(hostname, 0), - NodeState.LOST, - null, null, null, null, 1, null, 0); - nodesUpdated.add(nodeReport); - roleHistory.onNodesUpdated(nodesUpdated); - - nodemap = roleHistory.cloneNodemap(); - int endSize = nodemap.size(); - // as even unused nodes are added to the list, we expect the map size to - // be >1 - assertTrue(startSize <= endSize); - assertNotNull(nodemap.get(hostname)); - assertFalse(nodemap.get(hostname).isOnline()); - - // add a failure of a node we've never head of - String newhost = "newhost"; - nodesUpdated = Arrays.asList( - NodeReport.newInstance( - NodeId.newInstance(newhost, 0), - NodeState.LOST, - null, null, null, null, 1, null, 0) - ); - roleHistory.onNodesUpdated(nodesUpdated); - - NodeMap nodemap2 = roleHistory.cloneNodemap(); - assertNotNull(nodemap2.get(newhost)); - assertFalse(nodemap2.get(newhost).isOnline()); - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryFindNodesForNewInstances.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryFindNodesForNewInstances.java deleted file mode 100644 index ece65ba..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryFindNodesForNewInstances.java +++ /dev/null @@ -1,177 +0,0 @@ -/* - * 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.slider.server.appmaster.model.history; - -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockFactory; -import org.apache.slider.server.appmaster.model.mock.MockRoleHistory; -import org.apache.slider.server.appmaster.state.ContainerOutcome; -import org.apache.slider.server.appmaster.state.NodeEntry; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.RoleHistory; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -/** - * Testing finding nodes for new instances. - * - * This stresses the non-AA codepath - */ -public class TestRoleHistoryFindNodesForNewInstances extends - BaseMockAppStateTest { - private static final Logger LOG = - LoggerFactory.getLogger(TestRoleHistoryFindNodesForNewInstances.class); - - public TestRoleHistoryFindNodesForNewInstances() throws BadConfigException { - } - - @Override - public String getTestName() { - return "TestFindNodesForNewInstances"; - } - - private NodeInstance age1Active4; - private NodeInstance age2Active2; - private NodeInstance age3Active0; - private NodeInstance age4Active1; - private NodeInstance age2Active0; - - private RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - - private RoleStatus roleStat; - private RoleStatus roleStat2; - - @Override - public void setup() throws Exception { - super.setup(); - - age1Active4 = nodeInstance(1, 4, 0, 0); - age2Active2 = nodeInstance(2, 2, 0, 1); - age3Active0 = nodeInstance(3, 0, 0, 0); - age4Active1 = nodeInstance(4, 1, 0, 0); - age2Active0 = nodeInstance(2, 0, 0, 0); - - roleHistory.insert(Arrays.asList(age2Active2, age2Active0, age4Active1, - age1Active4, age3Active0)); - roleHistory.buildRecentNodeLists(); - - roleStat = getRole0Status(); - roleStat2 = getRole2Status(); - } - - public List findNodes(int count) { - return findNodes(count, roleStat); - } - - public List findNodes(int count, RoleStatus roleStatus) { - List found = new ArrayList<>(); - for (int i = 0; i < count; i++) { - NodeInstance f = roleHistory.findRecentNodeForNewInstance(roleStatus); - if (f != null) { - found.add(f); - } - } - return found; - } - - //@Test - public void testFind1NodeR0() throws Throwable { - NodeInstance found = roleHistory.findRecentNodeForNewInstance(roleStat); - LOG.info("found: {}", found); - assertTrue(Arrays.asList(age3Active0).contains(found)); - } - - //@Test - public void testFind2NodeR0() throws Throwable { - NodeInstance found = roleHistory.findRecentNodeForNewInstance(roleStat); - LOG.info("found: {}", found); - assertTrue(Arrays.asList(age2Active0, age3Active0).contains(found)); - NodeInstance found2 = roleHistory.findRecentNodeForNewInstance(roleStat); - LOG.info("found: {}", found2); - assertTrue(Arrays.asList(age2Active0, age3Active0).contains(found2)); - assertNotEquals(found, found2); - } - - //@Test - public void testFind3NodeR0ReturnsNull() throws Throwable { - assertEquals(2, findNodes(2).size()); - NodeInstance found = roleHistory.findRecentNodeForNewInstance(roleStat); - assertNull(found); - } - - //@Test - public void testFindNodesOneEntry() throws Throwable { - List foundNodes = findNodes(4, roleStat2); - assertEquals(0, foundNodes.size()); - } - - //@Test - public void testFindNodesIndependent() throws Throwable { - assertEquals(2, findNodes(2).size()); - roleHistory.dump(); - assertEquals(0, findNodes(3, roleStat2).size()); - } - - //@Test - public void testFindNodesFallsBackWhenUsed() throws Throwable { - // mark age2 and active 0 as busy, expect a null back - age2Active0.get(getRole0Status().getKey()).onStartCompleted(); - assertNotEquals(0, age2Active0.getActiveRoleInstances(getRole0Status() - .getKey())); - age3Active0.get(getRole0Status().getKey()).onStartCompleted(); - assertNotEquals(0, age3Active0.getActiveRoleInstances(getRole0Status() - .getKey())); - NodeInstance found = roleHistory.findRecentNodeForNewInstance(roleStat); - if (found != null) { - LOG.info(found.toFullString()); - } - assertNull(found); - } - //@Test - public void testFindNodesSkipsFailingNode() throws Throwable { - // mark age2 and active 0 as busy, expect a null back - - NodeEntry entry0 = age2Active0.get(getRole0Status().getKey()); - entry0.containerCompleted( - false, - ContainerOutcome.Failed); - assertTrue(entry0.getFailed() > 0); - assertTrue(entry0.getFailedRecently() > 0); - entry0.containerCompleted( - false, - ContainerOutcome.Failed); - assertFalse(age2Active0.exceedsFailureThreshold(roleStat)); - // set failure to 1 - roleStat.getProviderRole().nodeFailureThreshold = 1; - // threshold is now exceeded - assertTrue(age2Active0.exceedsFailureThreshold(roleStat)); - - // get the role & expect age3 to be picked up, even though it is older - NodeInstance found = roleHistory.findRecentNodeForNewInstance(roleStat); - assertEquals(age3Active0, found); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryNIComparators.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryNIComparators.java deleted file mode 100644 index 4d4cf62..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryNIComparators.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * 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.slider.server.appmaster.model.history; - -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockFactory; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -/** - * Unit test to verify the comparators sort as expected. - */ -public class TestRoleHistoryNIComparators extends BaseMockAppStateTest { - - private NodeInstance age1Active4; - private NodeInstance age2Active2; - private NodeInstance age3Active0; - private NodeInstance age4Active1; - private NodeInstance empty = new NodeInstance("empty", MockFactory - .ROLE_COUNT); - private NodeInstance age6failing; - private NodeInstance age1failing; - - private List nodes; - private List nodesPlusEmpty; - private List allnodes; - - private RoleStatus role0Status; - - @Override - public void setup() throws Exception { - super.setup(); - - role0Status = getRole0Status(); - - age1Active4 = nodeInstance(1001, 4, 0, 0); - age2Active2 = nodeInstance(1002, 2, 0, 0); - age3Active0 = nodeInstance(1003, 0, 0, 0); - age4Active1 = nodeInstance(1004, 1, 0, 0); - age6failing = nodeInstance(1006, 0, 0, 0); - age1failing = nodeInstance(1001, 0, 0, 0); - - age6failing.get(role0Status.getKey()).setFailedRecently(2); - age1failing.get(role0Status.getKey()).setFailedRecently(1); - - nodes = Arrays.asList(age2Active2, age4Active1, age1Active4, age3Active0); - nodesPlusEmpty = Arrays.asList(age2Active2, age4Active1, age1Active4, - age3Active0, empty); - allnodes = Arrays.asList(age6failing, age2Active2, age4Active1, - age1Active4, age3Active0, age1failing); - } - - @Override - public String getTestName() { - return "TestNIComparators"; - } - - //@Test - public void testPreferred() throws Throwable { - Collections.sort(nodes, new NodeInstance.Preferred(role0Status.getKey())); - assertListEquals(nodes, Arrays.asList(age4Active1, age3Active0, - age2Active2, age1Active4)); - } - - /** - * The preferred sort still includes failures; up to next phase in process - * to handle that. - * @throws Throwable - */ - //@Test - public void testPreferredWithFailures() throws Throwable { - Collections.sort(allnodes, new NodeInstance.Preferred(role0Status - .getKey())); - assertEquals(allnodes.get(0), age6failing); - assertEquals(allnodes.get(1), age4Active1); - } - - //@Test - public void testPreferredComparatorDowngradesFailures() throws Throwable { - NodeInstance.Preferred preferred = new NodeInstance.Preferred(role0Status - .getKey()); - assertEquals(-1, preferred.compare(age6failing, age1failing)); - assertEquals(1, preferred.compare(age1failing, age6failing)); - } - - //@Test - public void testNewerThanNoRole() throws Throwable { - Collections.sort(nodesPlusEmpty, new NodeInstance.Preferred(role0Status - .getKey())); - assertListEquals(nodesPlusEmpty, Arrays.asList(age4Active1, age3Active0, - age2Active2, age1Active4, empty)); - } - - //@Test - public void testMoreActiveThan() throws Throwable { - - Collections.sort(nodes, new NodeInstance.MoreActiveThan(role0Status - .getKey())); - assertListEquals(nodes, Arrays.asList(age1Active4, age2Active2, - age4Active1, age3Active0)); - } - - //@Test - public void testMoreActiveThanEmpty() throws Throwable { - - Collections.sort(nodesPlusEmpty, new NodeInstance.MoreActiveThan( - role0Status.getKey())); - assertListEquals(nodesPlusEmpty, Arrays.asList(age1Active4, age2Active2, - age4Active1, age3Active0, empty)); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.java deleted file mode 100644 index c1fc28f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryOutstandingRequestTracker.java +++ /dev/null @@ -1,385 +0,0 @@ -/* - * 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.slider.server.appmaster.model.history; - -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; -import org.apache.hadoop.yarn.util.resource.Resources; -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.providers.PlacementPolicy; -import org.apache.slider.providers.ProviderRole; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockAppState; -import org.apache.slider.server.appmaster.model.mock.MockContainer; -import org.apache.slider.server.appmaster.model.mock.MockNodeId; -import org.apache.slider.server.appmaster.model.mock.MockPriority; -import org.apache.slider.server.appmaster.model.mock.MockResource; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.CancelSingleRequest; -import org.apache.slider.server.appmaster.operations.ContainerRequestOperation; -import org.apache.slider.server.appmaster.state.ContainerAllocationOutcome; -import org.apache.slider.server.appmaster.state.ContainerAllocationResults; -import org.apache.slider.server.appmaster.state.ContainerPriority; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.OutstandingRequest; -import org.apache.slider.server.appmaster.state.OutstandingRequestTracker; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -/** - * Test outstanding request tracker. - */ -public class TestRoleHistoryOutstandingRequestTracker extends - BaseMockAppStateTest { - private static final Logger LOG = - LoggerFactory.getLogger(TestRoleHistoryOutstandingRequestTracker.class); - - public static final String WORKERS_LABEL = "workers"; - private NodeInstance host1 = new NodeInstance("host1", 3); - private NodeInstance host2 = new NodeInstance("host2", 3); - private MockResource resource = factory.newResource(48, 1); - - private OutstandingRequestTracker tracker = new OutstandingRequestTracker(); - - public static final String WORKER = "worker"; - - @Override - public Application buildApplication() { - Application application = super.buildApplication(); - Component component = new Component().name("worker").numberOfContainers(0L); - component.getConfiguration().setProperty(ResourceKeys.YARN_LABEL_EXPRESSION, - WORKERS_LABEL); - application.getComponents().add(component); - return application; - } - - //@Test - public void testAddRetrieveEntry() throws Throwable { - OutstandingRequest request = tracker.newRequest(host1, 0); - assertEquals(tracker.lookupPlacedRequest(0, "host1"), request); - assertEquals(tracker.removePlacedRequest(request), request); - assertNull(tracker.lookupPlacedRequest(0, "host1")); - } - - //@Test - public void testAddCompleteEntry() throws Throwable { - OutstandingRequest req1 = tracker.newRequest(host1, 0); - req1.buildContainerRequest(resource, getRole0Status(), 0); - - tracker.newRequest(host2, 0).buildContainerRequest(resource, - getRole0Status(), 0); - tracker.newRequest(host1, 1).buildContainerRequest(resource, - getRole0Status(), 0); - - ContainerAllocationResults allocation = tracker.onContainerAllocated(1, - "host1", null); - assertEquals(allocation.outcome, ContainerAllocationOutcome.Placed); - assertTrue(allocation.operations.get(0) instanceof CancelSingleRequest); - - assertNull(tracker.lookupPlacedRequest(1, "host1")); - assertNotNull(tracker.lookupPlacedRequest(0, "host1")); - } - - //@Test - public void testResetOpenRequests() throws Throwable { - OutstandingRequest req1 = tracker.newRequest(null, 0); - assertFalse(req1.isLocated()); - tracker.newRequest(host1, 0); - List openRequests = tracker.listOpenRequests(); - assertEquals(1, openRequests.size()); - tracker.resetOutstandingRequests(0); - assertTrue(tracker.listOpenRequests().isEmpty()); - assertTrue(tracker.listPlacedRequests().isEmpty()); - } - - //@Test - public void testRemoveOpenRequestUnissued() throws Throwable { - OutstandingRequest req1 = tracker.newRequest(null, 0); - req1.buildContainerRequest(resource, getRole0Status(), 0); - assertEquals(1, tracker.listOpenRequests().size()); - MockContainer c1 = factory.newContainer(null, new MockPriority(0)); - c1.setResource(resource); - - ContainerAllocationResults allocation = - tracker.onContainerAllocated(0, "host1", c1); - ContainerAllocationOutcome outcome = allocation.outcome; - assertEquals(outcome, ContainerAllocationOutcome.Unallocated); - assertTrue(allocation.operations.isEmpty()); - assertEquals(1, tracker.listOpenRequests().size()); - } - - //@Test - public void testIssuedOpenRequest() throws Throwable { - OutstandingRequest req1 = tracker.newRequest(null, 0); - req1.buildContainerRequest(resource, getRole0Status(), 0); - assertEquals(1, tracker.listOpenRequests().size()); - - int pri = ContainerPriority.buildPriority(0, false); - assertTrue(pri > 0); - MockNodeId nodeId = factory.newNodeId("hostname-1"); - MockContainer c1 = factory.newContainer(nodeId, new MockPriority(pri)); - - c1.setResource(resource); - - ContainerRequest issued = req1.getIssuedRequest(); - assertEquals(issued.getCapability(), resource); - assertEquals(issued.getPriority().getPriority(), c1.getPriority() - .getPriority()); - assertTrue(req1.resourceRequirementsMatch(resource)); - - ContainerAllocationResults allocation = - tracker.onContainerAllocated(0, nodeId.getHost(), c1); - assertEquals(0, tracker.listOpenRequests().size()); - assertTrue(allocation.operations.get(0) instanceof CancelSingleRequest); - - assertEquals(allocation.outcome, ContainerAllocationOutcome.Open); - assertEquals(allocation.origin, req1); - } - - //@Test - public void testResetEntries() throws Throwable { - tracker.newRequest(host1, 0); - tracker.newRequest(host2, 0); - tracker.newRequest(host1, 1); - List canceled = tracker.resetOutstandingRequests(0); - assertEquals(2, canceled.size()); - assertTrue(canceled.contains(host1)); - assertTrue(canceled.contains(host2)); - assertNotNull(tracker.lookupPlacedRequest(1, "host1")); - assertNull(tracker.lookupPlacedRequest(0, "host1")); - canceled = tracker.resetOutstandingRequests(0); - assertEquals(0, canceled.size()); - assertEquals(1, tracker.resetOutstandingRequests(1).size()); - } - - //@Test - public void testEscalation() throws Throwable { - // first request: default placement - assertEquals(getRole0Status().getPlacementPolicy(), PlacementPolicy - .DEFAULT); - Resource res0 = newResource(getRole0Status()); - OutstandingRequest outstanding0 = tracker.newRequest(host1, - getRole0Status().getKey()); - ContainerRequest initialRequest = - outstanding0.buildContainerRequest(res0, getRole0Status(), 0); - assertNotNull(outstanding0.getIssuedRequest()); - assertTrue(outstanding0.isLocated()); - assertFalse(outstanding0.isEscalated()); - assertFalse(initialRequest.getRelaxLocality()); - assertEquals(1, tracker.listPlacedRequests().size()); - - // second. This one doesn't get launched. This is to verify that the - // escalation process skips entries which are in the list but have not - // been issued, which can be a race condition between request issuance & - // escalation. - // (not one observed outside test authoring, but retained for completeness) - Resource res2 = newResource(getRole2Status()); - OutstandingRequest outstanding2 = tracker.newRequest(host1, - getRole2Status().getKey()); - - // simulate some time escalation of role 1 MUST now be triggered - long interval = getRole0Status().getPlacementTimeoutSeconds() * 1000 + 500; - long now = interval; - final List escalations = tracker - .escalateOutstandingRequests(now); - - assertTrue(outstanding0.isEscalated()); - assertFalse(outstanding2.isEscalated()); - - // two entries - assertEquals(2, escalations.size()); - AbstractRMOperation e1 = escalations.get(0); - assertTrue(e1 instanceof CancelSingleRequest); - final CancelSingleRequest cancel = (CancelSingleRequest) e1; - assertEquals(initialRequest, cancel.getRequest()); - AbstractRMOperation e2 = escalations.get(1); - assertTrue(e2 instanceof ContainerRequestOperation); - ContainerRequestOperation escRequest = (ContainerRequestOperation) e2; - assertTrue(escRequest.getRequest().getRelaxLocality()); - - // build that second request from an anti-affine entry - // these get placed as well - now += interval; - ContainerRequest containerReq2 = - outstanding2.buildContainerRequest(res2, getRole2Status(), now); - // escalate a little bit more - final List escalations2 = tracker - .escalateOutstandingRequests(now); - // and expect no new entries - assertEquals(0, escalations2.size()); - - // go past the role2 timeout - now += getRole2Status().getPlacementTimeoutSeconds() * 1000 + 500; - // escalate a little bit more - final List escalations3 = tracker - .escalateOutstandingRequests(now); - // and expect another escalation - assertEquals(2, escalations3.size()); - assertTrue(outstanding2.isEscalated()); - - // finally add a strict entry to the mix - Resource res3 = newResource(getRole1Status()); - OutstandingRequest outstanding3 = tracker.newRequest(host1, - getRole1Status().getKey()); - - final ProviderRole providerRole1 = getRole1Status().getProviderRole(); - assertEquals(providerRole1.placementPolicy, PlacementPolicy.STRICT); - now += interval; - assertFalse(outstanding3.mayEscalate()); - final List escalations4 = tracker - .escalateOutstandingRequests(now); - assertTrue(escalations4.isEmpty()); - - } - - /** - * If the placement does include a label, the initial request must - * not include it. - * The escalation request will contain the label, while - * leaving out the node list. - * retains the node list, but sets relaxLocality==true - * @throws Throwable - */ - //@Test - public void testRequestLabelledPlacement() throws Throwable { - NodeInstance ni = new NodeInstance("host1", 0); - OutstandingRequest req1 = tracker.newRequest(ni, 0); - Resource res0 = factory.newResource(48, 1); - - RoleStatus workerRole = lookupRole(WORKER); - // initial request - ContainerRequest yarnRequest = - req1.buildContainerRequest(res0, workerRole, 0); - assertEquals(req1.label, WORKERS_LABEL); - - assertNull(yarnRequest.getNodeLabelExpression()); - assertFalse(yarnRequest.getRelaxLocality()); - // escalation - ContainerRequest yarnRequest2 = req1.escalate(); - assertNull(yarnRequest2.getNodes()); - assertTrue(yarnRequest2.getRelaxLocality()); - assertEquals(yarnRequest2.getNodeLabelExpression(), WORKERS_LABEL); - } - - /** - * If the placement doesnt include a label, then the escalation request - * retains the node list, but sets relaxLocality==true. - * @throws Throwable - */ - //@Test - public void testRequestUnlabelledPlacement() throws Throwable { - NodeInstance ni = new NodeInstance("host1", 0); - OutstandingRequest req1 = tracker.newRequest(ni, 0); - Resource res0 = factory.newResource(48, 1); - - // initial request - ContainerRequest yarnRequest = req1.buildContainerRequest(res0, - getRole0Status(), 0); - assertNotNull(yarnRequest.getNodes()); - assertTrue(SliderUtils.isUnset(yarnRequest.getNodeLabelExpression())); - assertFalse(yarnRequest.getRelaxLocality()); - ContainerRequest yarnRequest2 = req1.escalate(); - assertNotNull(yarnRequest2.getNodes()); - assertTrue(yarnRequest2.getRelaxLocality()); - } - - //@Test(expected = IllegalArgumentException.class) - public void testAARequestNoNodes() throws Throwable { - tracker.newAARequest(getRole0Status().getKey(), new ArrayList<>(), ""); - } - - //@Test - public void testAARequest() throws Throwable { - int role0 = getRole0Status().getKey(); - OutstandingRequest request = tracker.newAARequest(role0, Arrays - .asList(host1), ""); - assertEquals(host1.hostname, request.hostname); - assertFalse(request.isLocated()); - } - - //@Test - public void testAARequestPair() throws Throwable { - int role0 = getRole0Status().getKey(); - OutstandingRequest request = tracker.newAARequest(role0, Arrays.asList( - host1, host2), ""); - assertEquals(host1.hostname, request.hostname); - assertFalse(request.isLocated()); - ContainerRequest yarnRequest = request.buildContainerRequest( - getRole0Status().copyResourceRequirements(new MockResource(0, 0)), - getRole0Status(), - 0); - assertFalse(yarnRequest.getRelaxLocality()); - assertFalse(request.mayEscalate()); - - assertEquals(2, yarnRequest.getNodes().size()); - } - - //@Test - public void testBuildResourceRequirements() throws Throwable { - // Store original values - Application application = appState.getClusterStatus(); - Component role0 = application.getComponent(getRole0Status().getName()); - String origMem = role0.getResource().getMemory(); - Integer origVcores = role0.getResource().getCpus(); - - // Resource values to be used for this test - int testMem = 32768; - int testVcores = 2; - role0.resource(new org.apache.slider.api.resource.Resource().memory(Integer - .toString(testMem)).cpus(testVcores)); - - // Test normalization disabled - LOG.info("Test normalization: disabled"); - role0.getConfiguration().setProperty( - ResourceKeys.YARN_RESOURCE_NORMALIZATION_ENABLED, "false"); - MockResource requestedRes = new MockResource(testMem, testVcores); - MockResource expectedRes = new MockResource(testMem, testVcores); - LOG.info("Resource requested: {}", requestedRes); - Resource resFinal = appState.buildResourceRequirements(getRole0Status()); - LOG.info("Resource actual: {}", resFinal); - assertTrue(Resources.equals(expectedRes, resFinal)); - - // Test normalization enabled - LOG.info("Test normalization: enabled"); - role0.getConfiguration().setProperty( - ResourceKeys.YARN_RESOURCE_NORMALIZATION_ENABLED, "true"); - expectedRes = new MockResource(MockAppState.RM_MAX_RAM, testVcores); - LOG.info("Resource requested: {}", requestedRes); - resFinal = appState.buildResourceRequirements(getRole0Status()); - LOG.info("Resource actual: {}", resFinal); - assertTrue(Resources.equals(expectedRes, resFinal)); - - // revert resource configuration to original value - role0.resource(new org.apache.slider.api.resource.Resource().memory(origMem) - .cpus(origVcores)); - } - - public Resource newResource(RoleStatus r) { - return appState.buildResourceRequirements(r); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.java deleted file mode 100644 index e3770a5..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRW.java +++ /dev/null @@ -1,371 +0,0 @@ -/* - * 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.slider.server.appmaster.model.history; - -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.Path; -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.providers.PlacementPolicy; -import org.apache.slider.providers.ProviderRole; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockFactory; -import org.apache.slider.server.appmaster.model.mock.MockRoleHistory; -import org.apache.slider.server.appmaster.state.NodeEntry; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.RoleHistory; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.apache.slider.server.avro.LoadedRoleHistory; -import org.apache.slider.server.avro.RoleHistoryWriter; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -/** - * Test fole history reading and writing. - */ -public class TestRoleHistoryRW extends BaseMockAppStateTest { - private static final Logger LOG = - LoggerFactory.getLogger(TestRoleHistoryRW.class); - - private static long time = System.currentTimeMillis(); - public static final String HISTORY_V1_6_ROLE = - "org/apache/slider/server/avro/history-v01-6-role.json"; - public static final String HISTORY_V1_3_ROLE = - "org/apache/slider/server/avro/history-v01-3-role.json"; - public static final String HISTORY_V1B_1_ROLE = - "org/apache/slider/server/avro/history_v01b_1_role.json"; - - private RoleStatus role0Status; - private RoleStatus role1Status; - - static final ProviderRole PROVIDER_ROLE3 = new ProviderRole( - "role3", - 3, - PlacementPolicy.STRICT, - 3, - 3, - ResourceKeys.DEF_YARN_LABEL_EXPRESSION); - - @Override - public String getTestName() { - return "TestHistoryRW"; - } - - @Override - public void setup() throws Exception { - super.setup(); - role0Status = getRole0Status(); - role1Status = getRole1Status(); - } - - //@Test - public void testWriteReadEmpty() throws Throwable { - RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - roleHistory.onStart(fs, historyPath); - Path history = roleHistory.saveHistory(time++); - assertTrue(fs.getFileStatus(history).isFile()); - RoleHistoryWriter historyWriter = new RoleHistoryWriter(); - historyWriter.read(fs, history); - } - - //@Test - public void testWriteReadData() throws Throwable { - RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - assertFalse(roleHistory.onStart(fs, historyPath)); - String addr = "localhost"; - NodeInstance instance = roleHistory.getOrCreateNodeInstance(addr); - NodeEntry ne1 = instance.getOrCreate(0); - ne1.setLastUsed(0xf00d); - - Path history = roleHistory.saveHistory(time++); - assertTrue(fs.getFileStatus(history).isFile()); - RoleHistoryWriter historyWriter = new RoleHistoryWriter(); - RoleHistory rh2 = new MockRoleHistory(MockFactory.ROLES); - - - LoadedRoleHistory loadedRoleHistory = historyWriter.read(fs, history); - assertTrue(0 < loadedRoleHistory.size()); - rh2.rebuild(loadedRoleHistory); - NodeInstance ni2 = rh2.getExistingNodeInstance(addr); - assertNotNull(ni2); - NodeEntry ne2 = ni2.get(0); - assertNotNull(ne2); - assertEquals(ne2.getLastUsed(), ne1.getLastUsed()); - } - - //@Test - public void testWriteReadActiveData() throws Throwable { - RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - roleHistory.onStart(fs, historyPath); - String addr = "localhost"; - String addr2 = "rack1server5"; - NodeInstance localhost = roleHistory.getOrCreateNodeInstance(addr); - NodeEntry orig1 = localhost.getOrCreate(role0Status.getKey()); - orig1.setLastUsed(0x10); - NodeInstance rack1server5 = roleHistory.getOrCreateNodeInstance(addr2); - NodeEntry orig2 = rack1server5.getOrCreate(role1Status.getKey()); - orig2.setLive(3); - assertFalse(orig2.isAvailable()); - NodeEntry orig3 = localhost.getOrCreate(role1Status.getKey()); - orig3.setLastUsed(0x20); - orig3.setLive(1); - assertFalse(orig3.isAvailable()); - orig3.release(); - assertTrue(orig3.isAvailable()); - roleHistory.dump(); - - long savetime = 0x0001000; - Path history = roleHistory.saveHistory(savetime); - assertTrue(fs.getFileStatus(history).isFile()); - describe("Loaded"); - LOG.info("testWriteReadActiveData in {}", history); - RoleHistoryWriter historyWriter = new RoleHistoryWriter(); - RoleHistory rh2 = new MockRoleHistory(MockFactory.ROLES); - LoadedRoleHistory loadedRoleHistory = historyWriter.read(fs, history); - assertEquals(3, loadedRoleHistory.size()); - rh2.rebuild(loadedRoleHistory); - rh2.dump(); - - assertEquals(2, rh2.getClusterSize()); - NodeInstance ni2 = rh2.getExistingNodeInstance(addr); - assertNotNull(ni2); - NodeEntry loadedNE = ni2.get(role0Status.getKey()); - assertEquals(loadedNE.getLastUsed(), orig1.getLastUsed()); - NodeInstance ni2b = rh2.getExistingNodeInstance(addr2); - assertNotNull(ni2b); - NodeEntry loadedNE2 = ni2b.get(role1Status.getKey()); - assertNotNull(loadedNE2); - assertEquals(loadedNE2.getLastUsed(), savetime); - assertEquals(rh2.getThawedDataTime(), savetime); - - // now start it - rh2.buildRecentNodeLists(); - describe("starting"); - rh2.dump(); - List available0 = rh2.cloneRecentNodeList(role0Status - .getKey()); - assertEquals(1, available0.size()); - - NodeInstance entry = available0.get(0); - assertEquals(entry.hostname, "localhost"); - assertEquals(entry, localhost); - List available1 = rh2.cloneRecentNodeList(role1Status - .getKey()); - assertEquals(2, available1.size()); - //and verify that even if last used was set, the save time is picked up - assertEquals(entry.get(role1Status.getKey()).getLastUsed(), roleHistory - .getSaveTime()); - - } - - //@Test - public void testWriteThaw() throws Throwable { - RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - assertFalse(roleHistory.onStart(fs, historyPath)); - String addr = "localhost"; - NodeInstance instance = roleHistory.getOrCreateNodeInstance(addr); - NodeEntry ne1 = instance.getOrCreate(0); - ne1.setLastUsed(0xf00d); - - Path history = roleHistory.saveHistory(time++); - long savetime =roleHistory.getSaveTime(); - assertTrue(fs.getFileStatus(history).isFile()); - RoleHistory rh2 = new MockRoleHistory(MockFactory.ROLES); - assertTrue(rh2.onStart(fs, historyPath)); - NodeInstance ni2 = rh2.getExistingNodeInstance(addr); - assertNotNull(ni2); - NodeEntry ne2 = ni2.get(0); - assertNotNull(ne2); - assertEquals(ne2.getLastUsed(), ne1.getLastUsed()); - assertEquals(rh2.getThawedDataTime(), savetime); - } - - - //@Test - public void testPurgeOlderEntries() throws Throwable { - RoleHistoryWriter historyWriter = new RoleHistoryWriter(); - time = 1; - Path file1 = touch(historyWriter, time++); - Path file2 = touch(historyWriter, time++); - Path file3 = touch(historyWriter, time++); - Path file4 = touch(historyWriter, time++); - Path file5 = touch(historyWriter, time++); - Path file6 = touch(historyWriter, time++); - - assertEquals(0, historyWriter.purgeOlderHistoryEntries(fs, file1)); - assertEquals(1, historyWriter.purgeOlderHistoryEntries(fs, file2)); - assertEquals(0, historyWriter.purgeOlderHistoryEntries(fs, file2)); - assertEquals(3, historyWriter.purgeOlderHistoryEntries(fs, file5)); - assertEquals(1, historyWriter.purgeOlderHistoryEntries(fs, file6)); - try { - // make an impossible assertion that will fail if the method - // actually completes - assertEquals(-1, historyWriter.purgeOlderHistoryEntries(fs, file1)); - } catch (FileNotFoundException ignored) { - // expected - } - - } - - public Path touch(RoleHistoryWriter historyWriter, long timeMs) - throws IOException { - Path path = historyWriter.createHistoryFilename(historyPath, timeMs); - FSDataOutputStream out = fs.create(path); - out.close(); - return path; - } - - //@Test - public void testSkipEmptyFileOnRead() throws Throwable { - describe("verify that empty histories are skipped on read; old histories " + - "purged"); - RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - roleHistory.onStart(fs, historyPath); - time = 0; - Path oldhistory = roleHistory.saveHistory(time++); - - String addr = "localhost"; - NodeInstance instance = roleHistory.getOrCreateNodeInstance(addr); - NodeEntry ne1 = instance.getOrCreate(0); - ne1.setLastUsed(0xf00d); - - Path goodhistory = roleHistory.saveHistory(time++); - - RoleHistoryWriter historyWriter = new RoleHistoryWriter(); - Path touched = touch(historyWriter, time++); - - RoleHistory rh2 = new MockRoleHistory(MockFactory.ROLES); - assertTrue(rh2.onStart(fs, historyPath)); - NodeInstance ni2 = rh2.getExistingNodeInstance(addr); - assertNotNull(ni2); - - //and assert the older file got purged - assertFalse(fs.exists(oldhistory)); - assertTrue(fs.exists(goodhistory)); - assertTrue(fs.exists(touched)); - } - - //@Test - public void testSkipBrokenFileOnRead() throws Throwable { - describe("verify that empty histories are skipped on read; old histories " + - "purged"); - RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - roleHistory.onStart(fs, historyPath); - time = 0; - Path oldhistory = roleHistory.saveHistory(time++); - - String addr = "localhost"; - NodeInstance instance = roleHistory.getOrCreateNodeInstance(addr); - NodeEntry ne1 = instance.getOrCreate(0); - ne1.setLastUsed(0xf00d); - - Path goodhistory = roleHistory.saveHistory(time++); - - RoleHistoryWriter historyWriter = new RoleHistoryWriter(); - Path badfile = historyWriter.createHistoryFilename(historyPath, time++); - FSDataOutputStream out = fs.create(badfile); - out.writeBytes("{broken:true}"); - out.close(); - - RoleHistory rh2 = new MockRoleHistory(MockFactory.ROLES); - describe("IGNORE STACK TRACE BELOW"); - - assertTrue(rh2.onStart(fs, historyPath)); - - describe("IGNORE STACK TRACE ABOVE"); - NodeInstance ni2 = rh2.getExistingNodeInstance(addr); - assertNotNull(ni2); - - //and assert the older file got purged - assertFalse(fs.exists(oldhistory)); - assertTrue(fs.exists(goodhistory)); - assertTrue(fs.exists(badfile)); - } - - /** - * Test that a v1 JSON file can be read. Here the number of roles - * matches the current state. - * @throws Throwable - */ - //@Test - public void testReloadDataV13Role() throws Throwable { - String source = HISTORY_V1_3_ROLE; - RoleHistoryWriter writer = new RoleHistoryWriter(); - - LoadedRoleHistory loadedRoleHistory = writer.read(source); - assertEquals(4, loadedRoleHistory.size()); - RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - assertEquals(0, roleHistory.rebuild(loadedRoleHistory)); - } - - /** - * Test that a v1 JSON file can be read. Here more roles than expected - * @throws Throwable - */ - //@Test - public void testReloadDataV16Role() throws Throwable { - String source = HISTORY_V1_6_ROLE; - RoleHistoryWriter writer = new RoleHistoryWriter(); - - LoadedRoleHistory loadedRoleHistory = writer.read(source); - assertEquals(6, loadedRoleHistory.size()); - RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - assertEquals(3, roleHistory.rebuild(loadedRoleHistory)); - } - - /** - * Test that a v1 JSON file can be read. Here the number of roles - * is less than the current state. - * @throws Throwable - */ - //@Test - public void testReloadLessRoles() throws Throwable { - String source = HISTORY_V1_3_ROLE; - RoleHistoryWriter writer = new RoleHistoryWriter(); - - LoadedRoleHistory loadedRoleHistory = writer.read(source); - assertEquals(4, loadedRoleHistory.size()); - List expandedRoles = new ArrayList(MockFactory.ROLES); - expandedRoles.add(PROVIDER_ROLE3); - RoleHistory roleHistory = new MockRoleHistory(expandedRoles); - assertEquals(0, roleHistory.rebuild(loadedRoleHistory)); - } - - /** - * Test that a v1b JSON file can be read. Here more roles than expected - * @throws Throwable - */ - //@Test - public void testReloadDataV1B1Role() throws Throwable { - String source = HISTORY_V1B_1_ROLE; - RoleHistoryWriter writer = new RoleHistoryWriter(); - - LoadedRoleHistory loadedRoleHistory = writer.read(source); - assertEquals(1, loadedRoleHistory.size()); - assertEquals(2, loadedRoleHistory.roleMap.size()); - RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - assertEquals(0, roleHistory.rebuild(loadedRoleHistory)); - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRWOrdering.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRWOrdering.java deleted file mode 100644 index 033b509..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRWOrdering.java +++ /dev/null @@ -1,162 +0,0 @@ -/* - * 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.slider.server.appmaster.model.history; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockFactory; -import org.apache.slider.server.appmaster.model.mock.MockRoleHistory; -import org.apache.slider.server.appmaster.state.NodeEntry; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.RoleHistory; -import org.apache.slider.server.avro.NewerFilesFirst; -import org.apache.slider.server.avro.RoleHistoryWriter; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.net.URI; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -/** - * Test role history rw ordering. - */ -public class TestRoleHistoryRWOrdering extends BaseMockAppStateTest { - private static final Logger LOG = - LoggerFactory.getLogger(TestRoleHistoryRWOrdering.class); - - private List paths = pathlist( - Arrays.asList( - "hdfs://localhost/history-0406c.json", - "hdfs://localhost/history-5fffa.json", - "hdfs://localhost/history-0001a.json", - "hdfs://localhost/history-0001f.json" - ) - ); - private Path h0406c = paths.get(0); - private Path h5fffa = paths.get(1); - private Path h0001a = paths.get(3); - - public TestRoleHistoryRWOrdering() throws URISyntaxException { - } - - List pathlist(List pathnames) throws URISyntaxException { - List pathList = new ArrayList<>(); - for (String p : pathnames) { - pathList.add(new Path(new URI(p))); - } - return pathList; - } - - @Override - public String getTestName() { - return "TestHistoryRWOrdering"; - } - - /** - * This tests regexp pattern matching. It uses the current time so isn't - * repeatable -but it does test a wider range of values in the process - * @throws Throwable - */ - //@Test - public void testPatternRoundTrip() throws Throwable { - describe("test pattern matching of names"); - long value=System.currentTimeMillis(); - String name = String.format(SliderKeys.HISTORY_FILENAME_CREATION_PATTERN, - value); - String matchpattern = SliderKeys.HISTORY_FILENAME_MATCH_PATTERN; - Pattern pattern = Pattern.compile(matchpattern); - Matcher matcher = pattern.matcher(name); - if (!matcher.find()) { - throw new Exception("No match for pattern $matchpattern in $name"); - } - } - - //@Test - public void testWriteSequenceReadData() throws Throwable { - describe("test that if multiple entries are written, the newest is picked" + - " up"); - long time = System.currentTimeMillis(); - - RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - assertFalse(roleHistory.onStart(fs, historyPath)); - String addr = "localhost"; - NodeInstance instance = roleHistory.getOrCreateNodeInstance(addr); - NodeEntry ne1 = instance.getOrCreate(0); - ne1.setLastUsed(0xf00d); - - Path history1 = roleHistory.saveHistory(time++); - Path history2 = roleHistory.saveHistory(time++); - Path history3 = roleHistory.saveHistory(time); - - //inject a later file with a different name - sliderFileSystem.cat(new Path(historyPath, "file.json"), true, "hello," + - " world"); - - - RoleHistoryWriter historyWriter = new RoleHistoryWriter(); - - List entries = historyWriter.findAllHistoryEntries( - fs, - historyPath, - false); - assertEquals(entries.size(), 3); - assertEquals(entries.get(0), history3); - assertEquals(entries.get(1), history2); - assertEquals(entries.get(2), history1); - } - - //@Test - public void testPathStructure() throws Throwable { - assertEquals(h5fffa.getName(), "history-5fffa.json"); - } - - //@Test - public void testPathnameComparator() throws Throwable { - - NewerFilesFirst newerName = new NewerFilesFirst(); - - LOG.info("{} name is {}", h5fffa, h5fffa.getName()); - LOG.info("{} name is {}", h0406c, h0406c.getName()); - assertEquals(newerName.compare(h5fffa, h5fffa), 0); - assertTrue(newerName.compare(h5fffa, h0406c) < 0); - assertTrue(newerName.compare(h5fffa, h0001a) < 0); - assertTrue(newerName.compare(h0001a, h5fffa) > 0); - - } - - //@Test - public void testPathSort() throws Throwable { - List paths2 = new ArrayList<>(paths); - RoleHistoryWriter.sortHistoryPaths(paths2); - assertListEquals(paths2, - Arrays.asList( - paths.get(1), - paths.get(0), - paths.get(3), - paths.get(2) - )); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.java deleted file mode 100644 index b84689c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryRequestTracking.java +++ /dev/null @@ -1,298 +0,0 @@ -/* - * 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.slider.server.appmaster.model.history; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.providers.PlacementPolicy; -import org.apache.slider.providers.ProviderRole; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockContainer; -import org.apache.slider.server.appmaster.model.mock.MockFactory; -import org.apache.slider.server.appmaster.model.mock.MockRoleHistory; -import org.apache.slider.server.appmaster.state.ContainerAllocationOutcome; -import org.apache.slider.server.appmaster.state.NodeEntry; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.OutstandingRequest; -import org.apache.slider.server.appmaster.state.RoleHistory; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Arrays; -import java.util.List; - -/** - * Test the RH availability list and request tracking: that hosts - * get removed and added. - */ -public class TestRoleHistoryRequestTracking extends BaseMockAppStateTest { - private static final Logger LOG = - LoggerFactory.getLogger(TestRoleHistoryRequestTracking.class); - - private String roleName = "test"; - - private NodeInstance age1Active4; - private NodeInstance age2Active2; - private NodeInstance age2Active0; - private NodeInstance age3Active0; - private NodeInstance age4Active1; - - private RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - // 1MB, 1 vcore - private Resource resource = Resource.newInstance(1, 1); - - private RoleStatus roleStatus; - - public TestRoleHistoryRequestTracking() throws BadConfigException { - } - - AMRMClient.ContainerRequest requestContainer(RoleStatus rs) { - return roleHistory.requestContainerForRole(rs).getIssuedRequest(); - } - - @Override - public String getTestName() { - return "TestRoleHistoryAvailableList"; - } - - @Override - public void setup() throws Exception { - super.setup(); - - age1Active4 = nodeInstance(1, 4, 0, 0); - age2Active2 = nodeInstance(2, 2, 0, 1); - age2Active0 = nodeInstance(2, 0, 0, 0); - age3Active0 = nodeInstance(3, 0, 0, 0); - age4Active1 = nodeInstance(4, 1, 0, 0); - - roleHistory.insert(Arrays.asList(age2Active2, age2Active0, age4Active1, - age1Active4, age3Active0)); - roleHistory.buildRecentNodeLists(); - roleStatus = getRole0Status(); - roleStatus.setResourceRequirements(Resource.newInstance(1, 1)); - } - - //@Test - public void testAvailableListBuiltForRoles() throws Throwable { - List available0 = roleHistory.cloneRecentNodeList( - roleStatus.getKey()); - assertListEquals(Arrays.asList(age3Active0, age2Active0), available0); - } - - //@Test - public void testRequestedNodeOffList() throws Throwable { - NodeInstance ni = roleHistory.findRecentNodeForNewInstance(roleStatus); - assertEquals(age3Active0, ni); - assertListEquals(Arrays.asList(age2Active0), - roleHistory.cloneRecentNodeList(roleStatus.getKey())); - roleHistory.requestInstanceOnNode(ni, - roleStatus, - resource - ); - } - - //@Test - public void testRequestedNodeOffListWithFailures() throws Throwable { - assertFalse(roleHistory.cloneRecentNodeList(roleStatus.getKey()).isEmpty()); - - NodeEntry age3role0 = recordAsFailed(age3Active0, roleStatus.getKey(), 4); - assertTrue(age3Active0.isConsideredUnreliable(roleStatus.getKey(), - roleStatus.getNodeFailureThreshold())); - recordAsFailed(age2Active0, roleStatus.getKey(), 4); - assertTrue(age2Active0.isConsideredUnreliable(roleStatus.getKey(), - roleStatus.getNodeFailureThreshold())); - // expect to get a null node back - NodeInstance ni = roleHistory.findRecentNodeForNewInstance(roleStatus); - assertNull(ni); - - // which is translated to a no-location request - AMRMClient.ContainerRequest req = roleHistory.requestInstanceOnNode(ni, - roleStatus, - resource).getIssuedRequest(); - - assertNull(req.getNodes()); - - LOG.info("resetting failure count"); - age3role0.resetFailedRecently(); - roleHistory.dump(); - assertEquals(0, age3role0.getFailedRecently()); - assertFalse(age3Active0.isConsideredUnreliable(roleStatus.getKey(), - roleStatus.getNodeFailureThreshold())); - assertFalse(roleHistory.cloneRecentNodeList(roleStatus.getKey()).isEmpty()); - // looking for a node should now find one - ni = roleHistory.findRecentNodeForNewInstance(roleStatus); - assertEquals(ni, age3Active0); - req = roleHistory.requestInstanceOnNode(ni, roleStatus, resource) - .getIssuedRequest(); - assertEquals(1, req.getNodes().size()); - } - - /** - * Verify that strict placement policies generate requests for nodes - * irrespective of their failed status. - * @throws Throwable - */ - //@Test - public void testStrictPlacementIgnoresFailures() throws Throwable { - - RoleStatus targetRole = getRole1Status(); - final ProviderRole providerRole1 = targetRole.getProviderRole(); - assertEquals(providerRole1.placementPolicy, PlacementPolicy.STRICT); - int key1 = targetRole.getKey(); - int key0 = getRole0Status().getKey(); - - List nodes0 = Arrays.asList(age1Active4, age2Active0, - age2Active2, age3Active0, age4Active1); - recordAllFailed(key0, 4, nodes0); - recordAllFailed(key1, 4, nodes0); - - // trigger a list rebuild - roleHistory.buildRecentNodeLists(); - List recentRole0 = roleHistory.cloneRecentNodeList(key0); - assertTrue(recentRole0.indexOf(age3Active0) < recentRole0 - .indexOf(age2Active0)); - - // the non-strict role has no suitable nodes - assertNull(roleHistory.findRecentNodeForNewInstance(getRole0Status())); - - - NodeInstance ni = roleHistory.findRecentNodeForNewInstance(targetRole); - assertNotNull(ni); - - NodeInstance ni2 = roleHistory.findRecentNodeForNewInstance(targetRole); - assertNotNull(ni2); - assertNotEquals(ni, ni2); - } - - //@Test - public void testFindAndRequestNode() throws Throwable { - AMRMClient.ContainerRequest req = requestContainer(roleStatus); - - assertEquals(age3Active0.hostname, req.getNodes().get(0)); - List a2 = roleHistory.cloneRecentNodeList(roleStatus - .getKey()); - assertListEquals(Arrays.asList(age2Active0), a2); - } - - //@Test - public void testRequestedNodeIntoReqList() throws Throwable { - requestContainer(roleStatus); - List requests = roleHistory.listPlacedRequests(); - assertEquals(1, requests.size()); - assertEquals(age3Active0.hostname, requests.get(0).hostname); - } - - //@Test - public void testCompletedRequestDropsNode() throws Throwable { - AMRMClient.ContainerRequest req = requestContainer(roleStatus); - List requests = roleHistory.listPlacedRequests(); - assertEquals(1, requests.size()); - String hostname = requests.get(0).hostname; - assertEquals(age3Active0.hostname, hostname); - assertEquals(hostname, req.getNodes().get(0)); - MockContainer container = factory.newContainer(req, hostname); - assertOnContainerAllocated(container, 2, 1); - assertNoOutstandingPlacedRequests(); - } - - public void assertOnContainerAllocated(Container c1, int p1, int p2) { - assertNotEquals(ContainerAllocationOutcome.Open, roleHistory - .onContainerAllocated(c1, p1, p2).outcome); - } - - public void assertOnContainerAllocationOpen(Container c1, int p1, int p2) { - assertEquals(ContainerAllocationOutcome.Open, roleHistory - .onContainerAllocated(c1, p1, p2).outcome); - } - - void assertNoOutstandingPlacedRequests() { - assertTrue(roleHistory.listPlacedRequests().isEmpty()); - } - - public void assertOutstandingPlacedRequests(int i) { - assertEquals(i, roleHistory.listPlacedRequests().size()); - } - - //@Test - public void testTwoRequests() throws Throwable { - AMRMClient.ContainerRequest req = requestContainer(roleStatus); - AMRMClient.ContainerRequest req2 = requestContainer(roleStatus); - List requests = roleHistory.listPlacedRequests(); - assertEquals(2, requests.size()); - MockContainer container = factory.newContainer(req, req.getNodes().get(0)); - assertOnContainerAllocated(container, 2, 1); - assertOutstandingPlacedRequests(1); - container = factory.newContainer(req2, req2.getNodes().get(0)); - assertOnContainerAllocated(container, 2, 2); - assertNoOutstandingPlacedRequests(); - } - - //@Test - public void testThreeRequestsOneUnsatisified() throws Throwable { - AMRMClient.ContainerRequest req = requestContainer(roleStatus); - AMRMClient.ContainerRequest req2 = requestContainer(roleStatus); - AMRMClient.ContainerRequest req3 = requestContainer(roleStatus); - List requests = roleHistory.listPlacedRequests(); - assertEquals(2, requests.size()); - MockContainer container = factory.newContainer(req, req.getNodes().get(0)); - assertOnContainerAllocated(container, 2, 1); - assertOutstandingPlacedRequests(1); - - container = factory.newContainer(req3, "three"); - assertOnContainerAllocationOpen(container, 3, 2); - assertOutstandingPlacedRequests(1); - - // the final allocation will trigger a cleanup - container = factory.newContainer(req2, "four"); - // no node dropped - assertEquals(ContainerAllocationOutcome.Unallocated, - roleHistory.onContainerAllocated(container, 3, 3).outcome); - // yet the list is now empty - assertNoOutstandingPlacedRequests(); - roleHistory.listOpenRequests().isEmpty(); - - // and the remainder goes onto the available list - List a2 = roleHistory.cloneRecentNodeList(roleStatus - .getKey()); - assertListEquals(Arrays.asList(age2Active0), a2); - } - - //@Test - public void testThreeRequests() throws Throwable { - AMRMClient.ContainerRequest req = requestContainer(roleStatus); - AMRMClient.ContainerRequest req2 = requestContainer(roleStatus); - AMRMClient.ContainerRequest req3 = requestContainer(roleStatus); - assertOutstandingPlacedRequests(2); - assertNull(req3.getNodes()); - MockContainer container = factory.newContainer(req, req.getNodes().get(0)); - assertOnContainerAllocated(container, 3, 1); - assertOutstandingPlacedRequests(1); - container = factory.newContainer(req2, req2.getNodes().get(0)); - assertOnContainerAllocated(container, 3, 2); - assertNoOutstandingPlacedRequests(); - container = factory.newContainer(req3, "three"); - assertOnContainerAllocationOpen(container, 3, 3); - assertNoOutstandingPlacedRequests(); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryUpdateBlacklist.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryUpdateBlacklist.java deleted file mode 100644 index a271859..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/history/TestRoleHistoryUpdateBlacklist.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * 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.slider.server.appmaster.model.history; - -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.server.appmaster.actions.ResetFailureWindow; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockAM; -import org.apache.slider.server.appmaster.model.mock.MockFactory; -import org.apache.slider.server.appmaster.model.mock.MockRMOperationHandler; -import org.apache.slider.server.appmaster.model.mock.MockRoleHistory; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.UpdateBlacklistOperation; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.RoleHistory; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Collection; -import java.util.List; - -/** - * Test updating blacklist. - */ -public class TestRoleHistoryUpdateBlacklist extends BaseMockAppStateTest { - private RoleHistory roleHistory = new MockRoleHistory(MockFactory.ROLES); - private Collection roleStatuses; - private RoleStatus roleStatus; - private NodeInstance ni; - - public TestRoleHistoryUpdateBlacklist() throws BadConfigException { - } - - @Override - public String getTestName() { - return "TestUpdateBlacklist"; - } - - @Override - public void setup() throws Exception { - super.setup(); - ni = nodeInstance(1, 0, 0, 0); - roleHistory.insert(Arrays.asList(ni)); - roleHistory.buildRecentNodeLists(); - appState.setRoleHistory(roleHistory); - roleStatus = getRole0Status(); - roleStatuses = Arrays.asList(roleStatus); - } - - //@Test - public void testUpdateBlacklist() { - assertFalse(ni.isBlacklisted()); - - // at threshold, blacklist is unmodified - recordAsFailed(ni, roleStatus.getKey(), MockFactory.NODE_FAILURE_THRESHOLD); - UpdateBlacklistOperation op = roleHistory.updateBlacklist(roleStatuses); - assertNull(op); - assertFalse(ni.isBlacklisted()); - - // threshold is reached, node goes on blacklist - recordAsFailed(ni, roleStatus.getKey(), 1); - op = roleHistory.updateBlacklist(roleStatuses); - assertNotNull(op); - assertTrue(ni.isBlacklisted()); - - // blacklist remains unmodified - op = roleHistory.updateBlacklist(roleStatuses); - assertNull(op); - assertTrue(ni.isBlacklisted()); - - // failure threshold reset, node goes off blacklist - ni.resetFailedRecently(); - op = roleHistory.updateBlacklist(roleStatuses); - assertNotNull(op); - assertFalse(ni.isBlacklisted()); - } - - //@Test - public void testBlacklistOperations() - throws Exception { - recordAsFailed(ni, roleStatus.getKey(), MockFactory - .NODE_FAILURE_THRESHOLD + 1); - - List ops = appState.reviewRequestAndReleaseNodes(); - assertListLength(ops, 1); - AbstractRMOperation op = ops.get(0); - assertTrue(op instanceof UpdateBlacklistOperation); - assertTrue(ni.isBlacklisted()); - - MockRMOperationHandler handler = new MockRMOperationHandler(); - assertEquals(0, handler.getBlacklisted()); - handler.execute(ops); - assertEquals(1, handler.getBlacklisted()); - - ResetFailureWindow resetter = new ResetFailureWindow(handler); - resetter.execute(new MockAM(), null, appState); - assertEquals(0, handler.getBlacklisted()); - assertFalse(ni.isBlacklisted()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/Allocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/Allocator.java deleted file mode 100644 index 419f2fb..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/Allocator.java +++ /dev/null @@ -1,123 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.slider.common.tools.SliderUtils; - -/** - * Provides allocation services to a cluster -both random and placed. - * - * Important: container allocations need an app attempt ID put into the - * container ID - */ -public class Allocator { - - private final MockYarnCluster cluster; - /** - * Rolling index into the cluster used for the next "random" assignment. - */ - private int rollingIndex = 0; - - Allocator(MockYarnCluster cluster) { - this.cluster = cluster; - } - - /** - * Allocate a node using the list of nodes in the container as the - * hints. - * @param request request - * @return the allocated container -or null for none - */ - MockContainer allocate(AMRMClient.ContainerRequest request) { - MockYarnCluster.MockYarnClusterNode node = null; - MockYarnCluster.MockYarnClusterContainer allocated = null; - if (SliderUtils.isNotEmpty(request.getNodes())) { - for (String host : request.getNodes()) { - node = cluster.lookup(host); - allocated = node.allocate(); - if (allocated != null) { - break; - } - } - } - - if (allocated != null) { - return createContainerRecord(request, allocated, node); - } else { - if (request.getRelaxLocality() || request.getNodes().isEmpty()) { - // fallback to anywhere - return allocateRandom(request); - } else { - //no match and locality can't be requested - return null; - } - } - } - - /** - * Allocate a node without any positioning -use whatever policy this allocator - * chooses. - * @param request request - * @return the allocated container -or null for none - */ - MockContainer allocateRandom(AMRMClient.ContainerRequest request) { - int start = rollingIndex; - MockYarnCluster.MockYarnClusterNode node = cluster.nodeAt(rollingIndex); - MockYarnCluster.MockYarnClusterContainer allocated = node.allocate(); - // if there is no space, try again -but stop when all the nodes - // have failed - while (allocated == null && start != nextIndex()) { - node = cluster.nodeAt(rollingIndex); - allocated = node.allocate(); - } - - //here the allocation is set, so create the response - return createContainerRecord(request, allocated, node); - } - - /** - * Create a container record -if one was allocated. - * @param allocated allocation -may be null - * @param node node with the container - * @return a container record, or null if there was no allocation - */ - public MockContainer createContainerRecord( - AMRMClient.ContainerRequest request, - MockYarnCluster.MockYarnClusterContainer allocated, - MockYarnCluster.MockYarnClusterNode node) { - if (allocated == null) { - // no space - return null; - } - MockContainer container = new MockContainer(); - container.setId(new MockContainerId(allocated.getCid())); - container.setNodeId(node.getNodeId()); - container.setNodeHttpAddress(node.httpAddress()); - container.setPriority(request.getPriority()); - container.setResource(request.getCapability()); - return container; - } - - public int nextIndex() { - rollingIndex = (rollingIndex + 1) % cluster.getClusterSize(); - return rollingIndex; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.java deleted file mode 100644 index 9632265..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/BaseMockAppStateTest.java +++ /dev/null @@ -1,539 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerState; -import org.apache.hadoop.yarn.api.records.ContainerStatus; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.slider.api.resource.Application; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.BadClusterStateException; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.core.exceptions.SliderInternalStateException; -import org.apache.slider.core.exceptions.TriggerClusterTeardownException; -import org.apache.slider.core.main.LauncherExitCodes; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.CancelSingleRequest; -import org.apache.slider.server.appmaster.operations.ContainerReleaseOperation; -import org.apache.slider.server.appmaster.operations.ContainerRequestOperation; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.AppStateBindingInfo; -import org.apache.slider.server.appmaster.state.ContainerAssignment; -import org.apache.slider.server.appmaster.state.ContainerOutcome; -import org.apache.slider.server.appmaster.state.NodeEntry; -import org.apache.slider.server.appmaster.state.NodeInstance; -import org.apache.slider.server.appmaster.state.NodeMap; -import org.apache.slider.server.appmaster.state.ProviderAppState; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.apache.slider.server.appmaster.state.StateAccessForProviders; -import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; -import org.apache.slider.utils.SliderTestBase; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.net.URI; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Locale; -import java.util.Map.Entry; - -/** - * Base for app state tests. - */ -public abstract class BaseMockAppStateTest extends SliderTestBase implements - MockRoles { - private static final Logger LOG = - LoggerFactory.getLogger(BaseMockAppStateTest.class); - protected static final List EMPTY_ID_LIST = Collections - .emptyList(); - - protected final MockFactory factory = MockFactory.INSTANCE; - protected MockAppState appState; - protected MockYarnEngine engine; - protected FileSystem fs; - protected SliderFileSystem sliderFileSystem; - protected File historyWorkDir; - protected Path historyPath; - protected MockApplicationId applicationId; - protected MockApplicationAttemptId applicationAttemptId; - protected StateAccessForProviders stateAccess; - - /** - * Override point: called in setup() to create the YARN engine; can - * be changed for different sizes and options. - * @return - */ - public MockYarnEngine createYarnEngine() { - return new MockYarnEngine(8, 8); - } - - @Override - public void setup() throws Exception { - super.setup(); - YarnConfiguration conf = SliderUtils.createConfiguration(); - fs = FileSystem.get(new URI("file:///"), conf); - sliderFileSystem = new SliderFileSystem(fs, conf); - engine = createYarnEngine(); - initApp(); - } - - /** - * Initialize the application. - * This uses the binding information supplied by {@link #buildBindingInfo()}. - */ - protected void initApp() - throws IOException, BadConfigException, BadClusterStateException { - String historyDirName = getTestName(); - applicationId = new MockApplicationId(1, 0); - applicationAttemptId = new MockApplicationAttemptId(applicationId, 1); - - historyWorkDir = new File("target/history", historyDirName); - historyPath = new Path(historyWorkDir.toURI()); - fs.delete(historyPath, true); - appState = new MockAppState(buildBindingInfo()); - stateAccess = new ProviderAppState(getValidTestName(), appState); - } - - /** - * Build the binding info from the default constructor values, - * the roles from {@link #factory}, and an instance definition. - * from {@link #buildApplication()} ()} - * @return - */ - protected AppStateBindingInfo buildBindingInfo() throws IOException { - AppStateBindingInfo binding = new AppStateBindingInfo(); - binding.application = buildApplication(); - ServiceApiUtil.validateAndResolveApplication(binding.application, - sliderFileSystem, SliderUtils.createConfiguration()); - //binding.roles = new ArrayList<>(factory.ROLES); - binding.fs = fs; - binding.historyPath = historyPath; - binding.nodeReports = engine.getNodeReports(); - return binding; - } - - /** - * Override point, define the instance definition. - * @return the instance definition - */ - public Application buildApplication() { - return factory.newApplication(0, 0, 0).name(getValidTestName()); - } - - /** - * Get the test name ... defaults to method name - * @return the method name - */ - public String getTestName() { - return methodName.getMethodName(); - } - - public String getValidTestName() { - return getTestName().toLowerCase(Locale.ENGLISH); - } - - public RoleStatus getRole0Status() { - return lookupRole(ROLE0); - } - - public RoleStatus lookupRole(String role) { - return appState.lookupRoleStatus(role); - } - - public RoleStatus getRole1Status() { - return lookupRole(ROLE1); - } - - public RoleStatus getRole2Status() { - return lookupRole(ROLE2); - } - - /** - * Build a role instance from a container assignment. - * @param assigned - * @return the instance - */ - public RoleInstance roleInstance(ContainerAssignment assigned) { - Container target = assigned.container; - RoleInstance failedInstance = - assigned.role.getProviderRole().failedInstances.poll(); - RoleInstance ri; - if (failedInstance != null) { - ri = new RoleInstance(target, failedInstance); - } else { - ri = new RoleInstance(target, assigned.role.getProviderRole()); - } - ri.roleId = assigned.role.getPriority(); - ri.role = assigned.role.getName(); - return ri; - } - - public NodeInstance nodeInstance(long age, int live0, int live1, int live2) { - NodeInstance ni = new NodeInstance(String.format("age%d-[%d,%d,%d]", age, - live0, live1, live2), MockFactory.ROLE_COUNT); - ni.getOrCreate(getRole0Status().getKey()).setLastUsed(age); - ni.getOrCreate(getRole0Status().getKey()).setLive(live0); - if (live1 > 0) { - ni.getOrCreate(getRole1Status().getKey()).setLive(live1); - } - if (live2 > 0) { - ni.getOrCreate(getRole2Status().getKey()).setLive(live2); - } - return ni; - } - - /** - * Create a container status event. - * @param c container - * @return a status - */ - ContainerStatus containerStatus(Container c) { - return containerStatus(c.getId()); - } - - /** - * Create a container status instance for the given ID, declaring - * that it was shut down by the application itself. - * @param cid container Id - * @return the instance - */ - public ContainerStatus containerStatus(ContainerId cid) { - ContainerStatus status = containerStatus(cid, - LauncherExitCodes.EXIT_CLIENT_INITIATED_SHUTDOWN); - return status; - } - - public ContainerStatus containerStatus(ContainerId cid, int exitCode) { - ContainerStatus status = ContainerStatus.newInstance( - cid, - ContainerState.COMPLETE, - "", - exitCode); - return status; - } - - /** - * Create nodes and bring them to the started state. - * @return a list of roles - */ - protected List createAndStartNodes() - throws TriggerClusterTeardownException, SliderInternalStateException { - return createStartAndStopNodes(new ArrayList<>()); - } - - /** - * Create, Start and stop nodes. - * @param completionResults List filled in with the status on all completed - * nodes - * @return the nodes - */ - public List createStartAndStopNodes( - List completionResults) - throws TriggerClusterTeardownException, SliderInternalStateException { - List released = new ArrayList<>(); - List instances = createAndSubmitNodes(released); - processSubmissionOperations(instances, completionResults, released); - return instances; - } - - /** - * Process the start/stop operations. - * @param instances - * @param completionResults - * @param released - */ - public void processSubmissionOperations( - List instances, - List completionResults, - List released) { - for (RoleInstance instance : instances) { - LOG.debug("Started {} on {}", instance.role, instance.id); - assertNotNull(appState.onNodeManagerContainerStarted(instance - .getContainerId())); - } - releaseContainers(completionResults, - released, - ContainerState.COMPLETE, - "released", - 0 - ); - } - - /** - * Release a list of containers, updating the completion results. - * @param completionResults - * @param containerIds - * @param containerState - * @param exitText - * @param containerExitCode - * @return - */ - public void releaseContainers( - List completionResults, - List containerIds, - ContainerState containerState, - String exitText, - int containerExitCode) { - for (ContainerId id : containerIds) { - ContainerStatus status = ContainerStatus.newInstance(id, - containerState, - exitText, - containerExitCode); - completionResults.add(appState.onCompletedContainer(status)); - } - } - - /** - * Create nodes and submit them. - * @return a list of roles - */ - public List createAndSubmitNodes() - throws TriggerClusterTeardownException, SliderInternalStateException { - return createAndSubmitNodes(new ArrayList<>()); - } - - /** - * Create nodes and submit them. - * @return a list of roles - */ - public List createAndSubmitNodes(List containerIds) - throws TriggerClusterTeardownException, SliderInternalStateException { - return createAndSubmitNodes(containerIds, new ArrayList<>()); - } - - /** - * Create nodes and submit them. - * @return a list of roles allocated - */ - public List createAndSubmitNodes( - List containerIds, - List operationsOut) - throws TriggerClusterTeardownException, SliderInternalStateException { - List ops = appState.reviewRequestAndReleaseNodes(); - return submitOperations(ops, containerIds, operationsOut); - } - - public List submitOperations( - List operationsIn, - List released) { - return submitOperations(operationsIn, released, new ArrayList<>()); - } - - /** - * Process the RM operations and send onContainersAllocated - * events to the app state. - * @param operationsIn list of incoming ops - * @param released released containers - * @return list of outbound operations - */ - public List submitOperations( - List operationsIn, - List released, - List operationsOut) { - List allocatedContainers = engine.execute(operationsIn, - released); - List assignments = new ArrayList<>(); - appState.onContainersAllocated(allocatedContainers, assignments, - operationsOut); - - List roles = new ArrayList<>(); - for (ContainerAssignment assigned : assignments) { - Container container = assigned.container; - RoleInstance ri = roleInstance(assigned); - //tell the app it arrived - LOG.debug("Start submitted {} on ${}", ri.role, container.getId()); - appState.containerStartSubmitted(container, ri); - roles.add(ri); - } - return roles; - } - - /** - * Add the AM to the app state. - */ - protected void addAppMastertoAppState() { -// appState.buildAppMasterNode( -// new MockContainerId(applicationAttemptId, 999999L), -// "appmaster", -// 0, -// null); - } - - /** - * Extract the list of container IDs from the list of role instances. - * @param instances instance list - * @param role role to look up - * @return the list of CIDs - */ - public List extractContainerIds( - List instances, - String role) { - List ids = new ArrayList<>(); - for (RoleInstance ri : instances) { - if (ri.role.equals(role)) { - ids.add(ri.getContainerId()); - } - } - return ids; - } - - /** - * Record a node as failing. - * @param node - * @param id - * @param count - * @return the entry - */ - public NodeEntry recordAsFailed(NodeInstance node, int id, int count) { - NodeEntry entry = node.getOrCreate(id); - for (int i = 1; i <= count; i++) { - entry.containerCompleted( - false, - ContainerOutcome.Failed); - } - return entry; - } - - protected void recordAllFailed(int id, int count, List nodes) { - for (NodeInstance node : nodes) { - recordAsFailed(node, id, count); - } - } - - /** - * Get the container request of an indexed entry. Includes some assertions - * for better diagnostics - * @param ops operation list - * @param index index in the list - * @return the request. - */ - public AMRMClient.ContainerRequest getRequest(List ops, - int index) { - assertTrue(index < ops.size()); - AbstractRMOperation op = ops.get(index); - assertTrue(op instanceof ContainerRequestOperation); - return ((ContainerRequestOperation) op).getRequest(); - } - - /** - * Get the cancel request of an indexed entry. Includes some assertions for - * better diagnostics - * @param ops operation list - * @param index index in the list - * @return the request. - */ - public AMRMClient.ContainerRequest getCancel(List ops, - int index) { - assertTrue(index < ops.size()); - AbstractRMOperation op = ops.get(index); - assertTrue(op instanceof CancelSingleRequest); - return ((CancelSingleRequest) op).getRequest(); - } - - /** - * Get the single request of a list of operations; includes the check for - * the size. - * @param ops operations list of size 1 - * @return the request within the first ContainerRequestOperation - */ - public AMRMClient.ContainerRequest getSingleRequest( - List ops) { - assertEquals(1, ops.size()); - return getRequest(ops, 0); - } - - /** - * Get the single request of a list of operations; includes the check for - * the size. - * @param ops operations list of size 1 - * @return the request within the first operation - */ - public AMRMClient.ContainerRequest getSingleCancel( - List ops) { - assertEquals(1, ops.size()); - return getCancel(ops, 0); - } - - /** - * Get the single release of a list of operations; includes the check for - * the size. - * @param ops operations list of size 1 - * @return the request within the first operation - */ - public ContainerReleaseOperation getSingleRelease( - List ops) { - assertEquals(1, ops.size()); - AbstractRMOperation op = ops.get(0); - assertTrue(op instanceof ContainerReleaseOperation); - return (ContainerReleaseOperation) op; - } - - /** - * Get the node information as a large JSON String. - * @return - */ - protected String nodeInformationSnapshotAsString() - throws UnsupportedEncodingException, JsonProcessingException { - return prettyPrintAsJson(stateAccess.getNodeInformationSnapshot()); - } - - /** - * Scan through all containers and assert that the assignment is AA. - * @param index role index - */ - protected void assertAllContainersAA(int index) { - for (Entry nodeMapEntry : cloneNodemap().entrySet()) { - String name = nodeMapEntry.getKey(); - NodeInstance ni = nodeMapEntry.getValue(); - NodeEntry nodeEntry = ni.get(index); - assertTrue("too many instances on node " + name, nodeEntry == null || - nodeEntry.isAntiAffinityConstraintHeld()); - } - } - - /** - * Get a snapshot of the nodemap of the application state. - * @return a cloned nodemap - */ - protected NodeMap cloneNodemap() { - return appState.getRoleHistory().cloneNodemap(); - } - - /** - * Issue a nodes updated event. - * @param report report to notify - * @return response of AM - */ - protected AppState.NodeUpdatedOutcome updateNodes(NodeReport report) { - return appState.onNodesUpdated(Collections.singletonList(report)); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockAM.java deleted file mode 100644 index 66ae0f9..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockAM.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.slider.server.appmaster.SliderAppMaster; - -/** - * Mock AM. - */ -public class MockAM extends SliderAppMaster { -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockAppState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockAppState.java deleted file mode 100644 index 2fcf054..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockAppState.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.slider.core.exceptions.BadClusterStateException; -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.providers.ProviderRole; -import org.apache.slider.server.appmaster.management.MetricsAndMonitoring; -import org.apache.slider.server.appmaster.state.AbstractClusterServices; -import org.apache.slider.server.appmaster.state.AppState; -import org.apache.slider.server.appmaster.state.AppStateBindingInfo; - -import java.io.IOException; -import java.util.Map; - -/** - * Extended app state that makes more things public. - */ -public class MockAppState extends AppState { - public static final int RM_MAX_RAM = 4096; - public static final int RM_MAX_CORES = 64; - - private long time = -1; - - public MockAppState(AbstractClusterServices recordFactory) { - super(recordFactory, new MetricsAndMonitoring()); - setContainerLimits(1, RM_MAX_RAM, 1, RM_MAX_CORES); - } - - /** - * Instance with a mock record factory. - */ - public MockAppState() { - this(new MockClusterServices()); - } - - public MockAppState(AppStateBindingInfo bindingInfo) - throws BadClusterStateException, IOException, BadConfigException { - this(); - buildInstance(bindingInfo); - } - - public Map getRoleMap() { - return super.getRoleMap(); - } - - /** - * Current time. if the time field - * is set, that value is returned - * @return the current time. - */ - protected long now() { - if (time > 0) { - return time; - } - return System.currentTimeMillis(); - } - - public void setTime(long newTime) { - this.time = newTime; - } - - public void incTime(long inc) { - this.time = this.time + inc; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockApplicationAttemptId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockApplicationAttemptId.java deleted file mode 100644 index b509625..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockApplicationAttemptId.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; - -class MockApplicationAttemptId extends ApplicationAttemptId { - - private ApplicationId applicationId; - private int attemptId; - - public MockApplicationAttemptId() { - } - - public MockApplicationAttemptId(ApplicationId applicationId, int attemptId) { - this.applicationId = applicationId; - this.attemptId = attemptId; - } - - @Override - public ApplicationId getApplicationId() { - return applicationId; - } - - @Override - public void setApplicationId(ApplicationId applicationId) { - this.applicationId = applicationId; - } - - @Override - public int getAttemptId() { - return attemptId; - } - - @Override - public void setAttemptId(int attemptId) { - this.attemptId = attemptId; - } - - @Override - protected void build() { - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockApplicationId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockApplicationId.java deleted file mode 100644 index 01da470..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockApplicationId.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.yarn.api.records.ApplicationId; - -/** - * Mock app id. - */ -public class MockApplicationId extends ApplicationId { - - private int id; - private long clusterTimestamp; - - public MockApplicationId() { - } - - public MockApplicationId(int id) { - this.id = id; - } - - public MockApplicationId(int id, long clusterTimestamp) { - this.id = id; - this.clusterTimestamp = clusterTimestamp; - } - - @Override - public int getId() { - return id; - } - - @Override - public void setId(int id) { - this.id = id; - } - - @Override - public long getClusterTimestamp() { - return clusterTimestamp; - } - - @Override - public void setClusterTimestamp(long clusterTimestamp) { - this.clusterTimestamp = clusterTimestamp; - } - - @Override - public void build() { - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockClusterServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockClusterServices.java deleted file mode 100644 index 2578595..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockClusterServices.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.slider.server.appmaster.state.AbstractClusterServices; - -/** - * Mock cluster services. - */ -public class MockClusterServices extends AbstractClusterServices { - - @Override - public Resource newResource() { - return new MockResource(0, 0); - } - - @Override - public Resource newResource(int memory, int cores) { - return new MockResource(memory, cores); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockContainer.java deleted file mode 100644 index 148b7f6..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockContainer.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ExecutionType; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.Token; - -/** - * Mock container. - */ -public class MockContainer extends Container { - - private ContainerId id; - private NodeId nodeId; - private String nodeHttpAddress; - private Resource resource; - private Priority priority; - private Token containerToken; - - @Override - public int compareTo(Container other) { - if (this.getId().compareTo(other.getId()) == 0) { - if (this.getNodeId().compareTo(other.getNodeId()) == 0) { - return this.getResource().compareTo(other.getResource()); - } else { - return this.getNodeId().compareTo(other.getNodeId()); - } - } else { - return this.getId().compareTo(other.getId()); - } - } - - @Override - public String toString() { - return "MockContainer{ id=" + id + - ", nodeHttpAddress='" + nodeHttpAddress + "'," + - " priority=" + priority + " }"; - } - - @Override - public ContainerId getId() { - return id; - } - - @Override - public void setId(ContainerId id) { - this.id = id; - } - - @Override - public NodeId getNodeId() { - return nodeId; - } - - @Override - public void setNodeId(NodeId nodeId) { - this.nodeId = nodeId; - } - - @Override - public String getNodeHttpAddress() { - return nodeHttpAddress; - } - - @Override - public void setNodeHttpAddress(String nodeHttpAddress) { - this.nodeHttpAddress = nodeHttpAddress; - } - - @Override - public Resource getResource() { - return resource; - } - - @Override - public void setResource(Resource resource) { - this.resource = resource; - } - - @Override - public Priority getPriority() { - return priority; - } - - @Override - public void setPriority(Priority priority) { - this.priority = priority; - } - - @Override - public Token getContainerToken() { - return containerToken; - } - - @Override - public void setContainerToken(Token containerToken) { - this.containerToken = containerToken; - } - - @Override - public ExecutionType getExecutionType() { - return null; - } - - @Override - public void setExecutionType(ExecutionType executionType) { - - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockContainerId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockContainerId.java deleted file mode 100644 index 3cbc7e5..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockContainerId.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ContainerId; - -/** - * Mock container id. - */ -public class MockContainerId extends ContainerId implements Cloneable { - - private static final MockApplicationAttemptId DEFAULT_APP_ATTEMPT_ID = - new MockApplicationAttemptId(new MockApplicationId(1), 1); - - private long containerId; - private ApplicationAttemptId applicationAttemptId; - - MockContainerId() { - } - - /** - * Sets up a default app Attempt ID. - * @param containerId - */ - MockContainerId(long containerId) { - this.containerId = containerId; - this.applicationAttemptId = DEFAULT_APP_ATTEMPT_ID; - } - - public MockContainerId(ApplicationAttemptId applicationAttemptId, - long containerId) { - this.containerId = containerId; - this.applicationAttemptId = applicationAttemptId; - } - - MockContainerId(ContainerId that) { - containerId = that.getContainerId(); - applicationAttemptId = that.getApplicationAttemptId(); - } - - @Deprecated - @Override - public int getId() { - return (int) containerId; - } - - // TODO: Temporarily adding it back - void setId(int id) { - containerId = (long) id; - } - - @Override - public long getContainerId() { - return this.containerId; - } - - @Override - public void setContainerId(long id) { - this.containerId = id; - } - - @Override - public ApplicationAttemptId getApplicationAttemptId() { - return applicationAttemptId; - } - - @Override - public void setApplicationAttemptId(ApplicationAttemptId - applicationAttemptId) { - this.applicationAttemptId = applicationAttemptId; - } - - @Override - public void build() { - - } - - @Override - public String toString() { - return "mockcontainer_" + containerId; - } - - @Override - protected Object clone() throws CloneNotSupportedException { - return super.clone(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockFactory.java deleted file mode 100644 index 8785b92..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockFactory.java +++ /dev/null @@ -1,273 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerState; -import org.apache.hadoop.yarn.api.records.ContainerStatus; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.NodeState; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.impl.pb.NodeReportPBImpl; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.slider.api.ResourceKeys; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Component; -import org.apache.slider.api.resource.Resource; -import org.apache.slider.providers.PlacementPolicy; -import org.apache.slider.providers.ProviderRole; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; - -import static org.apache.slider.api.ResourceKeys.COMPONENT_PLACEMENT_POLICY; - -/** - * Factory for creating things. - */ -public class MockFactory implements MockRoles { - - public static final int NODE_FAILURE_THRESHOLD = 2; - - public static final MockFactory INSTANCE = new MockFactory(); - - /** - * Basic role. - */ - public static final ProviderRole PROVIDER_ROLE0 = new ProviderRole( - MockRoles.ROLE0, - 0, - PlacementPolicy.DEFAULT, - NODE_FAILURE_THRESHOLD, - 1, - ResourceKeys.DEF_YARN_LABEL_EXPRESSION); - /** - * role 1 is strict. timeout should be irrelevant; same as failures - */ - public static final ProviderRole PROVIDER_ROLE1 = new ProviderRole( - MockRoles.ROLE1, - 1, - PlacementPolicy.STRICT, - NODE_FAILURE_THRESHOLD, - 1, - ResourceKeys.DEF_YARN_LABEL_EXPRESSION); - - /** - * role 2: longer delay. - */ - public static final ProviderRole PROVIDER_ROLE2 = new ProviderRole( - MockRoles.ROLE2, - 2, - PlacementPolicy.ANYWHERE, - NODE_FAILURE_THRESHOLD, - 2, - ResourceKeys.DEF_YARN_LABEL_EXPRESSION); - - /** - * Patch up a "role2" role to have anti-affinity set. - */ - public static final ProviderRole AAROLE_2 = new ProviderRole( - MockRoles.ROLE2, - 2, - PlacementPolicy.ANTI_AFFINITY_REQUIRED, - NODE_FAILURE_THRESHOLD, - 2, - null); - - /** - * Patch up a "role1" role to have anti-affinity set and GPI as the label. - */ - public static final ProviderRole AAROLE_1_GPU = new ProviderRole( - MockRoles.ROLE1, - 1, - PlacementPolicy.ANTI_AFFINITY_REQUIRED, - NODE_FAILURE_THRESHOLD, - 1, - MockRoles.LABEL_GPU); - - private int appIdCount; - private int attemptIdCount; - private int containerIdCount; - - private ApplicationId appId = newAppId(); - private ApplicationAttemptId attemptId = newApplicationAttemptId(appId); - - /** - * List of roles. - */ - public static final List ROLES = Arrays.asList( - PROVIDER_ROLE0, - PROVIDER_ROLE1, - PROVIDER_ROLE2 - ); - - public static final int ROLE_COUNT = ROLES.size(); - - MockContainerId newContainerId() { - return newContainerId(attemptId); - } - - MockContainerId newContainerId(ApplicationAttemptId attemptId0) { - MockContainerId cid = new MockContainerId(attemptId0, containerIdCount++); - return cid; - } - - MockApplicationAttemptId newApplicationAttemptId(ApplicationId appId0) { - MockApplicationAttemptId id = new MockApplicationAttemptId(appId0, - attemptIdCount++); - return id; - } - - MockApplicationId newAppId() { - MockApplicationId id = new MockApplicationId(); - id.setId(appIdCount++); - return id; - } - - public MockNodeId newNodeId(String host) { - return new MockNodeId(host); - } - - MockContainer newContainer(ContainerId cid) { - MockContainer c = new MockContainer(); - c.setId(cid); - return c; - } - - public MockContainer newContainer() { - return newContainer(newContainerId()); - } - - public MockContainer newContainer(NodeId nodeId, Priority priority) { - MockContainer container = newContainer(newContainerId()); - container.setNodeId(nodeId); - container.setPriority(priority); - return container; - } - - /** - * Build a new container using the request to supply priority and resource. - * @param req request - * @param host hostname to assign to - * @return the container - */ - public MockContainer newContainer(AMRMClient.ContainerRequest req, String - host) { - MockContainer container = newContainer(newContainerId()); - container.setResource(req.getCapability()); - container.setPriority(req.getPriority()); - container.setNodeId(new MockNodeId(host)); - return container; - } - - /** - * Create a new instance with the given components definined in the - * resources section. - * @param r1 - * @param r2 - * @param r3 - * @return - */ - public Application newApplication(long r1, long r2, long r3) { - Application application = new Application(); - application.setLaunchCommand("sleep 60"); - application.setResource(new Resource().memory("256")); - application.getConfiguration().setProperty(ResourceKeys - .NODE_FAILURE_THRESHOLD, Integer.toString(NODE_FAILURE_THRESHOLD)); - List components = application.getComponents(); - Component c1 = new Component().name(ROLE0).numberOfContainers(r1); - c1.getConfiguration().setProperty(COMPONENT_PLACEMENT_POLICY, - Integer.toString(PlacementPolicy.DEFAULT)); - Component c2 = new Component().name(ROLE1).numberOfContainers(r2); - c2.getConfiguration().setProperty(COMPONENT_PLACEMENT_POLICY, - Integer.toString(PlacementPolicy.STRICT)); - Component c3 = new Component().name(ROLE2).numberOfContainers(r3); - c3.getConfiguration().setProperty(COMPONENT_PLACEMENT_POLICY, - Integer.toString(PlacementPolicy.ANYWHERE)); - components.add(c1); - components.add(c2); - components.add(c3); - return application; - } - - public MockResource newResource(int memory, int vcores) { - return new MockResource(memory, vcores); - } - - ContainerStatus newContainerStatus() { - return newContainerStatus(null, null, "", 0); - } - - ContainerStatus newContainerStatus(ContainerId containerId, - ContainerState containerState, String diagnostics, int exitStatus) { - return ContainerStatus.newInstance(containerId, containerState, - diagnostics, exitStatus); - } - - /** - * Create a single instance. - * @param hostname - * @param nodeState - * @param label - */ - public NodeReport newNodeReport(String hostname, NodeState nodeState, - String label) { - NodeId nodeId = NodeId.newInstance(hostname, 80); - Integer.valueOf(hostname, 16); - return newNodeReport(hostname, nodeId, nodeState, label); - } - - NodeReport newNodeReport( - String hostname, - NodeId nodeId, - NodeState nodeState, - String label) { - NodeReport report = new NodeReportPBImpl(); - HashSet nodeLabels = new HashSet<>(); - nodeLabels.add(label); - report.setNodeId(nodeId); - report.setNodeLabels(nodeLabels); - report.setNodeState(nodeState); - report.setHttpAddress("http$hostname:80"); - return report; - } - - /** - * Create a list of instances -one for each hostname. - * @param hostnames hosts - * @return - */ - public List createNodeReports( - List hostnames, NodeState nodeState, String label) { - if (nodeState == null) { - nodeState = NodeState.RUNNING; - } - List reports = new ArrayList<>(); - for (String name : hostnames) { - reports.add(newNodeReport(name, nodeState, label)); - } - return reports; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockFileSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockFileSystem.java deleted file mode 100644 index 72d1665..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockFileSystem.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.fs.FilterFileSystem; -import org.apache.hadoop.fs.Path; - -import java.io.IOException; - -/** - * - */ -class MockFileSystem extends FilterFileSystem{ - @Override - public Path resolvePath(Path p) throws IOException { - return new Path("hdfs://localhost/", p); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockNodeId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockNodeId.java deleted file mode 100644 index 9d2379a..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockNodeId.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.yarn.api.records.NodeId; - -/** - * Mock node id. - */ -public class MockNodeId extends NodeId { - private String host; - private int port; - - public MockNodeId() { - } - - MockNodeId(String host) { - this.host = host; - } - - public MockNodeId(String host, int port) { - this.host = host; - this.port = port; - } - - public String getHost() { - return host; - } - - public void setHost(String host) { - this.host = host; - } - - public int getPort() { - return port; - } - - public void setPort(int port) { - this.port = port; - } - - @Override - protected void build() { - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockPriority.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockPriority.java deleted file mode 100644 index 36f97cc..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockPriority.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.yarn.api.records.Priority; - -/** - * Mock priority. - */ -public class MockPriority extends Priority { - - private int priority; - - public MockPriority(int priority) { - this.priority = priority; - } - - MockPriority() { - } - - @Override - public int getPriority() { - return priority; - } - - @Override - public void setPriority(int priority) { - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRMOperationHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRMOperationHandler.java deleted file mode 100644 index 3dd764a..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRMOperationHandler.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.ContainerReleaseOperation; -import org.apache.slider.server.appmaster.operations.ContainerRequestOperation; -import org.apache.slider.server.appmaster.operations.RMOperationHandler; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; - -/** - * Mock RM operation handler. - */ -public class MockRMOperationHandler extends RMOperationHandler { - protected static final Logger LOG = - LoggerFactory.getLogger(MockRMOperationHandler.class); - - private List operations = new ArrayList<>(); - private int requests; - private int releases; - // number available to cancel - private int availableToCancel = 0; - // count of cancelled values. This must be explicitly set - private int cancelled; - // number blacklisted - private int blacklisted = 0; - - @Override - public void releaseAssignedContainer(ContainerId containerId) { - operations.add(new ContainerReleaseOperation(containerId)); - LOG.info("Releasing container ID " + containerId.getContainerId()); - releases++; - } - - @Override - public void addContainerRequest(AMRMClient.ContainerRequest req) { - operations.add(new ContainerRequestOperation(req)); - LOG.info("Requesting container role #" + req.getPriority()); - requests++; - } - - @Override - public int cancelContainerRequests( - Priority priority1, - Priority priority2, - int count) { - int releaseable = Math.min(count, availableToCancel); - availableToCancel -= releaseable; - cancelled += releaseable; - return releaseable; - } - - @Override - public void cancelSingleRequest(AMRMClient.ContainerRequest request) { - // here assume that there is a copy of this request in the list - if (availableToCancel > 0) { - availableToCancel--; - cancelled++; - } - } - - @Override - public void updateBlacklist(List blacklistAdditions, List - blacklistRemovals) { - blacklisted += blacklistAdditions.size(); - blacklisted -= blacklistRemovals.size(); - } - - /** - * Clear the history. - */ - public void clear() { - operations.clear(); - releases = 0; - requests = 0; - } - - public AbstractRMOperation getFirstOp() { - return operations.get(0); - } - - public int getNumReleases() { - return releases; - } - - public void setAvailableToCancel(int num) { - this.availableToCancel = num; - } - - public int getAvailableToCancel() { - return availableToCancel; - } - - public int getBlacklisted() { - return blacklisted; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRecordFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRecordFactory.java deleted file mode 100644 index eb34586..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRecordFactory.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -/** - * Node report for testing. - */ -class MockRecordFactory { - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRegistryOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRegistryOperations.java deleted file mode 100644 index 4917f1b..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRegistryOperations.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.fs.PathNotFoundException; -import org.apache.hadoop.registry.client.api.RegistryOperations; -import org.apache.hadoop.registry.client.types.RegistryPathStatus; -import org.apache.hadoop.registry.client.types.ServiceRecord; -import org.apache.hadoop.service.AbstractService; - -import java.util.List; - -/** - * Simple stub registry for when one is needed for its API, but the operations - * are not actually required. - */ -class MockRegistryOperations extends AbstractService implements - RegistryOperations { - - MockRegistryOperations() { - super("mock"); - } - - @Override - public boolean mknode(String path, boolean createParents) { - return true; - } - - @Override - public void bind(String path, ServiceRecord record, int flags) { - } - - @Override - public ServiceRecord resolve(String path) throws PathNotFoundException { - throw new PathNotFoundException(path); - } - - @Override - public RegistryPathStatus stat(String path) throws PathNotFoundException { - throw new PathNotFoundException(path); - } - - @Override - public boolean exists(String path) { - return false; - } - - @Override - public List list(String path) throws PathNotFoundException { - throw new PathNotFoundException(path); - } - - @Override - public void delete(String path, boolean recursive) { - - } - - @Override - public boolean addWriteAccessor(String id, String pass) { - return true; - } - - @Override - public void clearWriteAccessors() { - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockResource.java deleted file mode 100644 index 3a2ccd7..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockResource.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.yarn.api.records.Resource; - -/** - * Mock resource. - */ -public class MockResource extends Resource { - private int memory; - private int virtualCores; - - public MockResource(int memory, int vcores) { - this.memory = memory; - this.virtualCores = vcores; - } - - @Override - public int compareTo(Resource other) { - long diff = this.getMemorySize() - other.getMemorySize(); - if (diff == 0) { - diff = this.getVirtualCores() - other.getVirtualCores(); - } - return diff == 0 ? 0 : (diff > 0 ? 1 : -1); - } - - @Override - public long getMemorySize() { - return memory; - } - - @Override - public void setMemorySize(long memorySize) { - memory = (int) memorySize; - } - - @Override - public int getVirtualCores() { - return virtualCores; - } - - @Override - public void setVirtualCores(int vCores) { - this.virtualCores = vCores; - } - - @Deprecated - @Override - public int getMemory() { - return memory; - } - - @Deprecated - @Override - public void setMemory(int memory) { - this.memory = memory; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRoleHistory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRoleHistory.java deleted file mode 100644 index 8e88b0d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRoleHistory.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.slider.core.exceptions.BadConfigException; -import org.apache.slider.providers.ProviderRole; -import org.apache.slider.server.appmaster.state.RoleHistory; -import org.apache.slider.server.appmaster.state.RoleStatus; - -import java.util.ArrayList; -import java.util.List; - -/** - * Subclass to enable access to some of the protected methods. - */ -public class MockRoleHistory extends RoleHistory { - - /** - * Take a list of provider roles and build the history from them, - * dynamically creating the role status entries on the way. - * @param providerRoles provider role list - * @throws BadConfigException configuration problem with the role list - */ - public MockRoleHistory(List providerRoles) throws - BadConfigException { - super(convertRoles(providerRoles), new MockClusterServices()); - } - - static List convertRoles(List providerRoles) { - List statuses = new ArrayList<>(); - for (ProviderRole role : providerRoles) { - statuses.add(new RoleStatus(role)); - } - return statuses; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRoles.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRoles.java deleted file mode 100644 index bad82bd..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockRoles.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -/** - * Mock role constants. - */ -public interface MockRoles { - - String ROLE0 = "role0"; - String ROLE1 = "role1"; - String ROLE2 = "role2"; - String LABEL_GPU = "gpu"; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockYarnCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockYarnCluster.java deleted file mode 100644 index 6b685a0..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockYarnCluster.java +++ /dev/null @@ -1,342 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.NodeState; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; -import java.util.Locale; - -/** - * Models the cluster itself: a set of mock cluster nodes. - * - * nodes retain the slot model with a limit of 2^8 slots/host -this - * lets us use 24 bits of the container ID for hosts, and so simulate - * larger hosts. - * - * upper 32: index into nodes in the cluster - * NodeID hostname is the index in hex format; this is parsed down to the index - * to resolve the host - * - * Important: container IDs will be reused as containers get recycled. This - * is not an attempt to realistically mimic a real YARN cluster, just - * simulate it enough for Slider to explore node re-use and its handling - * of successful and unsuccessful allocations. - * - * There is little or no checking of valid parameters in here -this is for - * test use, not production. - */ -public class MockYarnCluster { - protected static final Logger LOG = - LoggerFactory.getLogger(MockYarnCluster.class); - - private final int clusterSize; - private final int containersPerNode; - private MockYarnClusterNode[] nodes; - - MockYarnCluster(int clusterSize, int containersPerNode) { - this.clusterSize = clusterSize; - this.containersPerNode = containersPerNode; - build(); - } - - public int getClusterSize() { - return clusterSize; - } - - @Override - public String toString() { - return "MockYarnCluster size=" + clusterSize + ", capacity=" + - totalClusterCapacity()+ ", in use=" + containersInUse(); - } - - /** - * Build the cluster. - */ - private void build() { - nodes = new MockYarnClusterNode[clusterSize]; - for (int i = 0; i < clusterSize; i++) { - nodes[i] = new MockYarnClusterNode(i, containersPerNode); - } - } - - public MockYarnClusterNode nodeAt(int index) { - return nodes[index]; - } - - MockYarnClusterNode lookup(String hostname) { - int index = Integer.valueOf(hostname, 16); - return nodeAt(index); - } - - MockYarnClusterNode lookup(NodeId nodeId) { - return lookup(nodeId.getHost()); - } - - MockYarnClusterNode lookupOwner(ContainerId cid) { - return nodeAt(extractHost(cid.getContainerId())); - } - - /** - * Release a container: return true if it was actually in use. - * @param cid container ID - * @return the container released - */ - MockYarnClusterContainer release(ContainerId cid) { - int host = extractHost(cid.getContainerId()); - MockYarnClusterContainer inUse = nodeAt(host).release(cid.getContainerId()); - LOG.debug("Released {} inuse={}", cid, inUse); - return inUse; - } - - int containersInUse() { - int count = 0; - for (MockYarnClusterNode it : nodes) { - count += it.containersInUse(); - } - return count; - } - - /** - * Containers free. - * @return - */ - int containersFree() { - return totalClusterCapacity() - containersInUse(); - } - - int totalClusterCapacity() { - return clusterSize * containersPerNode; - } - - /** - * Reset all the containers. - */ - public void reset() { - for (MockYarnClusterNode node : nodes) { - node.reset(); - } - } - - /** - * Bulk allocate the specific number of containers on a range of the cluster. - * @param startNode start of the range - * @param endNode end of the range - * @param count count - * @return the number actually allocated -it will be less the count supplied - * if the node was full - */ - public int bulkAllocate(int startNode, int endNode, int count) { - int total = 0; - for (int i = startNode; i <= endNode; i++) { - total += nodeAt(i).bulkAllocate(count).size(); - } - return total; - } - - /** - * Get the list of node reports. These are not cloned; updates will persist - * in the nodemap - * @return current node report list - */ - List getNodeReports() { - List reports = new ArrayList<>(); - - for (MockYarnClusterNode n : nodes) { - reports.add(n.nodeReport); - } - return reports; - } - - /** - * Model cluster nodes on the simpler "slot" model than the YARN-era - * resource allocation model. Why? Easier to implement scheduling. - * Of course, if someone does want to implement the full process... - * - */ - public static class MockYarnClusterNode { - - private final int nodeIndex; - private final String hostname; - private List labels = new ArrayList<>(); - private final MockNodeId nodeId; - private final MockYarnClusterContainer[] containers; - private boolean offline; - private NodeReport nodeReport; - - public MockYarnClusterNode(int index, int size) { - nodeIndex = index; - hostname = String.format(Locale.ENGLISH, "%08x", index); - nodeId = new MockNodeId(hostname, 0); - - containers = new MockYarnClusterContainer[size]; - for (int i = 0; i < size; i++) { - int cid = makeCid(index, i); - MockContainerId mci = new MockContainerId(cid); - containers[i] = new MockYarnClusterContainer(mci); - } - - nodeReport = MockFactory.INSTANCE.newNodeReport(hostname, nodeId, - NodeState.RUNNING, ""); - } - - public String getHostname() { - return hostname; - } - - public NodeId getNodeId() { - return nodeId; - } - - /** - * Look up a container. - * @param containerId - * @return - */ - public MockYarnClusterContainer lookup(int containerId) { - return containers[extractContainer(containerId)]; - } - - /** - * Go offline; release all containers. - */ - public void goOffline() { - if (!offline) { - offline = true; - reset(); - } - } - - public void goOnline() { - offline = false; - } - - /** - * Allocate a container -if one is available. - * @return the container or null for none free - * -or the cluster node is offline - */ - public MockYarnClusterContainer allocate() { - if (!offline) { - for (int i = 0; i < containers.length; i++) { - MockYarnClusterContainer c = containers[i]; - if (!c.busy) { - c.busy = true; - return c; - } - } - } - return null; - } - - /** - * Bulk allocate the specific number of containers. - * @param count count - * @return the list actually allocated -it will be less the count supplied - * if the node was full - */ - public List bulkAllocate(int count) { - List result = new ArrayList<>(); - for (int i = 0; i < count; i++) { - MockYarnClusterContainer allocation = allocate(); - if (allocation == null) { - break; - } - result.add(allocation); - } - return result; - } - - /** - * Release a container. - * @param cid container ID - * @return the container if the container was busy before the release - */ - public MockYarnClusterContainer release(long cid) { - MockYarnClusterContainer container = containers[extractContainer(cid)]; - boolean b = container.busy; - container.busy = false; - return b? container: null; - } - - public String httpAddress() { - return "http://$hostname/"; - } - - /** - * Reset all the containers. - */ - public void reset() { - for (MockYarnClusterContainer cont : containers) { - cont.reset(); - } - } - - public int containersInUse() { - int c = 0; - for (MockYarnClusterContainer cont : containers) { - c += cont.busy ? 1 : 0; - } - return c; - } - - public int containersFree() { - return containers.length - containersInUse(); - } - } - - /** - * Cluster container. - */ - public static class MockYarnClusterContainer { - private MockContainerId cid; - private boolean busy; - - MockYarnClusterContainer(MockContainerId cid) { - this.cid = cid; - } - - public MockContainerId getCid() { - return cid; - } - - void reset() { - busy = false; - } - } - - public static int makeCid(int hostIndex, int containerIndex) { - return (hostIndex << 8) | containerIndex & 0xff; - } - - public static final int extractHost(long cid) { - return (int)((cid >>> 8) & 0xffff); - } - - public static final int extractContainer(long cid) { - return (int)(cid & 0xff); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockYarnEngine.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockYarnEngine.java deleted file mode 100644 index 9c5708f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/mock/MockYarnEngine.java +++ /dev/null @@ -1,188 +0,0 @@ -/* - * 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.slider.server.appmaster.model.mock; - -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.client.api.AMRMClient; -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; -import org.apache.slider.server.appmaster.operations.AbstractRMOperation; -import org.apache.slider.server.appmaster.operations.CancelSingleRequest; -import org.apache.slider.server.appmaster.operations.ContainerReleaseOperation; -import org.apache.slider.server.appmaster.operations.ContainerRequestOperation; -import org.junit.Assert; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; - -import static org.junit.Assert.assertNotNull; - -/** - * This is an evolving engine to mock YARN operations. - */ -public class MockYarnEngine { - private static final Logger LOG = - LoggerFactory.getLogger(MockYarnEngine.class); - - private MockYarnCluster cluster; - private Allocator allocator; - private List pending = new ArrayList<>(); - - private ApplicationId appId = new MockApplicationId(0, 0); - - private ApplicationAttemptId attemptId = new MockApplicationAttemptId(appId, - 1); - - @Override - public String toString() { - return "MockYarnEngine " + cluster + " + pending=" + pending.size(); - } - - public int containerCount() { - return cluster.containersInUse(); - } - - public MockYarnEngine(int clusterSize, int containersPerNode) { - cluster = new MockYarnCluster(clusterSize, containersPerNode); - allocator = new Allocator(cluster); - } - - public MockYarnCluster getCluster() { - return cluster; - } - - public Allocator getAllocator() { - return allocator; - } - - /** - * Allocate a container from a request. The containerID will be - * unique, nodeId and other fields chosen internally with - * no such guarantees; resource and priority copied over - * @param request request - * @return container - */ - public Container allocateContainer(AMRMClient.ContainerRequest request) { - MockContainer allocated = allocator.allocate(request); - if (allocated != null) { - MockContainerId id = (MockContainerId)allocated.getId(); - id.setApplicationAttemptId(attemptId); - } - return allocated; - } - - MockYarnCluster.MockYarnClusterContainer releaseContainer(ContainerId - containerId) { - return cluster.release(containerId); - } - - /** - * Process a list of operations -release containers to be released, - * allocate those for which there is space (but don't rescan the list after - * the scan). - * @param ops - * @return - */ - public List execute(List ops) { - return execute(ops, new ArrayList<>()); - } - - /** - * Process a list of operations -release containers to be released, - * allocate those for which there is space (but don't rescan the list after - * the scan). Unsatisifed entries are appended to the "pending" list - * @param ops operations - * @return the list of all satisfied operations - */ - public List execute(List ops, - List released) { - validateRequests(ops); - List allocation = new ArrayList<>(); - for (AbstractRMOperation op : ops) { - if (op instanceof ContainerReleaseOperation) { - ContainerReleaseOperation cro = (ContainerReleaseOperation) op; - ContainerId cid = cro.getContainerId(); - assertNotNull(releaseContainer(cid)); - released.add(cid); - } else if (op instanceof CancelSingleRequest) { - // no-op - LOG.debug("cancel request {}", op); - } else if (op instanceof ContainerRequestOperation) { - ContainerRequestOperation req = (ContainerRequestOperation) op; - Container container = allocateContainer(req.getRequest()); - if (container != null) { - LOG.info("allocated container {} for {}", container, req); - allocation.add(container); - } else { - LOG.debug("Unsatisfied allocation {}", req); - pending.add(req); - } - } else { - LOG.warn("Unsupported operation {}", op); - } - } - return allocation; - } - - /** - * Try and mimic some of the logic of AMRMClientImpl - * .checkLocalityRelaxationConflict. - * @param ops operations list - */ - void validateRequests(List ops) { - // run through the requests and verify that they are all consistent. - List outstandingRequests = new ArrayList<>(); - for (AbstractRMOperation operation : ops) { - if (operation instanceof ContainerRequestOperation) { - ContainerRequestOperation containerRequest = - (ContainerRequestOperation) operation; - ContainerRequest amRequest = containerRequest.getRequest(); - Priority priority = amRequest.getPriority(); - boolean relax = amRequest.getRelaxLocality(); - - for (ContainerRequestOperation req : outstandingRequests) { - if (req.getPriority() == priority && req.getRelaxLocality() != - relax) { - // mismatch in values - Assert.fail("operation " + operation + " has incompatible request" + - " priority from outsanding request"); - } - outstandingRequests.add(containerRequest); - - } - - } - } - } - - /** - * Get the list of node reports. These are not cloned; updates will persist - * in the nodemap. - * @return current node report list - */ - List getNodeReports() { - return cluster.getNodeReports(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/monkey/TestMockMonkey.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/monkey/TestMockMonkey.java deleted file mode 100644 index 16bd195..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/model/monkey/TestMockMonkey.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * 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.slider.server.appmaster.model.monkey; - -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.slider.api.InternalKeys; -import org.apache.slider.server.appmaster.actions.ActionHalt; -import org.apache.slider.server.appmaster.actions.ActionKillContainer; -import org.apache.slider.server.appmaster.actions.AsyncAction; -import org.apache.slider.server.appmaster.actions.QueueService; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockRMOperationHandler; -import org.apache.slider.server.appmaster.monkey.ChaosKillAM; -import org.apache.slider.server.appmaster.monkey.ChaosKillContainer; -import org.apache.slider.server.appmaster.monkey.ChaosMonkeyService; -import org.apache.slider.server.appmaster.monkey.ChaosTarget; -import org.apache.slider.server.appmaster.operations.ContainerReleaseOperation; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; -import java.util.concurrent.TimeUnit; - -/** - * Test chaos monkey. - */ -public class TestMockMonkey extends BaseMockAppStateTest { - private static final Logger LOG = - LoggerFactory.getLogger(TestMockMonkey.class); - - /** - * This queue service is NOT started; tests need to poll the queue - * rather than expect them to execute. - */ - private QueueService queues; - private ChaosMonkeyService monkey; - - @Before - public void init() { - YarnConfiguration configuration = new YarnConfiguration(); - queues = new QueueService(); - queues.init(configuration); - monkey = new ChaosMonkeyService(METRICS.getMetrics(), queues); - monkey.init(configuration); - } - - //@Test - public void testMonkeyStart() throws Throwable { - monkey.start(); - monkey.stop(); - } - - //@Test - public void testMonkeyPlay() throws Throwable { - ChaosCounter counter = new ChaosCounter(); - monkey.addTarget("target", counter, InternalKeys.PROBABILITY_PERCENT_100); - assertEquals(1, monkey.getTargetCount()); - monkey.play(); - assertEquals(1, counter.count); - } - - //@Test - public void testMonkeySchedule() throws Throwable { - ChaosCounter counter = new ChaosCounter(); - assertEquals(0, monkey.getTargetCount()); - monkey.addTarget("target", counter, InternalKeys.PROBABILITY_PERCENT_100); - assertEquals(1, monkey.getTargetCount()); - assertTrue(monkey.schedule(0, 1, TimeUnit.SECONDS)); - assertEquals(1, queues.scheduledActions.size()); - } - - //@Test - public void testMonkeyDoesntAddProb0Actions() throws Throwable { - ChaosCounter counter = new ChaosCounter(); - monkey.addTarget("target", counter, 0); - assertEquals(0, monkey.getTargetCount()); - monkey.play(); - assertEquals(0, counter.count); - } - - //@Test - public void testMonkeyScheduleProb0Actions() throws Throwable { - ChaosCounter counter = new ChaosCounter(); - monkey.addTarget("target", counter, 0); - assertFalse(monkey.schedule(0, 1, TimeUnit.SECONDS)); - assertEquals(0, queues.scheduledActions.size()); - } - - //@Test - public void testMonkeyPlaySometimes() throws Throwable { - ChaosCounter counter = new ChaosCounter(); - ChaosCounter counter2 = new ChaosCounter(); - monkey.addTarget("target1", counter, InternalKeys.PROBABILITY_PERCENT_1 - * 50); - monkey.addTarget("target2", counter2, InternalKeys - .PROBABILITY_PERCENT_1 * 25); - - for (int i = 0; i < 100; i++) { - monkey.play(); - } - LOG.info("Counter1 = {} counter2 = {}", counter.count, counter2.count); - /* - * Relying on probability here to give approximate answers - */ - assertTrue(counter.count > 25); - assertTrue(counter.count < 75); - assertTrue(counter2.count < counter.count); - } - - //@Test - public void testAMKiller() throws Throwable { - - ChaosKillAM chaos = new ChaosKillAM(queues, -1); - chaos.chaosAction(); - assertEquals(1, queues.scheduledActions.size()); - AsyncAction action = queues.scheduledActions.take(); - assertTrue(action instanceof ActionHalt); - } - - //@Test - public void testContainerKillerEmptyApp() throws Throwable { - - - ChaosKillContainer chaos = new ChaosKillContainer(appState, - queues, - new MockRMOperationHandler()); - chaos.chaosAction(); - assertEquals(0, queues.scheduledActions.size()); - } - - @Ignore - //@Test - public void testContainerKillerIgnoresAM() throws Throwable { - // TODO: AM needed in live container list? - addAppMastertoAppState(); - assertEquals(1, appState.getLiveContainers().size()); - - ChaosKillContainer chaos = new ChaosKillContainer(appState, - queues, - new MockRMOperationHandler()); - chaos.chaosAction(); - assertEquals(0, queues.scheduledActions.size()); - } - - //@Test - public void testContainerKiller() throws Throwable { - MockRMOperationHandler ops = new MockRMOperationHandler(); - getRole0Status().setDesired(1); - List instances = createAndStartNodes(); - assertEquals(1, instances.size()); - RoleInstance instance = instances.get(0); - - ChaosKillContainer chaos = new ChaosKillContainer(appState, queues, ops); - chaos.chaosAction(); - assertEquals(1, queues.scheduledActions.size()); - AsyncAction action = queues.scheduledActions.take(); - ActionKillContainer killer = (ActionKillContainer) action; - assertEquals(killer.getContainerId(), instance.getContainerId()); - killer.execute(null, queues, appState); - assertEquals(1, ops.getNumReleases()); - - ContainerReleaseOperation operation = (ContainerReleaseOperation) ops - .getFirstOp(); - assertEquals(operation.getContainerId(), instance.getContainerId()); - } - - /** - * Chaos target that just implements a counter. - */ - private static class ChaosCounter implements ChaosTarget { - private int count; - - @Override - public void chaosAction() { - count++; - } - - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder( - "ChaosCounter{"); - sb.append("count=").append(count); - sb.append('}'); - return sb.toString(); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/security/TestSecurityConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/security/TestSecurityConfiguration.java deleted file mode 100644 index 6e77806..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/security/TestSecurityConfiguration.java +++ /dev/null @@ -1,215 +0,0 @@ -/* - * 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.slider.server.appmaster.security; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.slider.api.resource.Application; -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; -import org.apache.slider.core.exceptions.SliderException; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** - * Test security configuration. - */ -public class TestSecurityConfiguration { - - //@Test - public void testValidLocalConfiguration() throws Throwable { - Configuration config = new Configuration(); - config.set(CommonConfigurationKeysPublic - .HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - Map compOps = new HashMap<>(); - compOps.put(SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL, "test"); - compOps.put(SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH, - "/some/local/path"); - Application application = new Application().configuration(new org.apache - .slider.api.resource.Configuration().properties(compOps)); - - SecurityConfiguration securityConfiguration = - new SecurityConfiguration(config, application, "testCluster"); - } - - //@Test - public void testValidDistributedConfiguration() throws Throwable { - Configuration config = new Configuration(); - config.set(CommonConfigurationKeysPublic - .HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - Map compOps = new HashMap<>(); - compOps.put(SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL, "test"); - compOps.put(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME, "some.keytab"); - Application application = new Application().configuration(new org.apache - .slider.api.resource.Configuration().properties(compOps)); - - SecurityConfiguration securityConfiguration = - new SecurityConfiguration(config, application, "testCluster"); - } - - //@Test - public void testMissingPrincipalNoLoginWithDistributedConfig() throws - Throwable { - Configuration config = new Configuration(); - config.set(CommonConfigurationKeysPublic - .HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - Map compOps = new HashMap<>(); - compOps.put(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME, "some.keytab"); - Application application = new Application().configuration(new org.apache - .slider.api.resource.Configuration().properties(compOps)); - - try { - SecurityConfiguration securityConfiguration = - new SecurityConfiguration(config, application, "testCluster") { - @Override - protected UserGroupInformation getLoginUser() throws - IOException { - return null; - } - }; - fail("expected SliderException"); - } catch (SliderException e) { - // expected - } - } - - //@Test - public void testMissingPrincipalNoLoginWithLocalConfig() throws Throwable { - Configuration config = new Configuration(); - config.set(CommonConfigurationKeysPublic - .HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - Map compOps = new HashMap<>(); - compOps.put(SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH, - "/some/local/path"); - Application application = new Application().configuration(new org.apache - .slider.api.resource.Configuration().properties(compOps)); - - try { - SecurityConfiguration securityConfiguration = - new SecurityConfiguration(config, application, "testCluster") { - @Override - protected UserGroupInformation getLoginUser() throws IOException { - return null; - } - }; - fail("expected SliderException"); - } catch (SliderException e) { - // expected - } - } - - //@Test - public void testBothKeytabMechanismsConfigured() throws Throwable { - Configuration config = new Configuration(); - config.set(CommonConfigurationKeysPublic - .HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - Map compOps = new HashMap<>(); - compOps.put(SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL, "test"); - compOps.put(SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH, - "/some/local/path"); - compOps.put(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME, "some.keytab"); - Application application = new Application().configuration(new org.apache - .slider.api.resource.Configuration().properties(compOps)); - - try { - SecurityConfiguration securityConfiguration = - new SecurityConfiguration(config, application, - "testCluster"); - fail("expected SliderException"); - } catch (SliderException e) { - // expected - } - } - - //@Test - public void testMissingPrincipalButLoginWithDistributedConfig() throws - Throwable { - Configuration config = new Configuration(); - config.set(CommonConfigurationKeysPublic - .HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - Map compOps = new HashMap<>(); - compOps.put(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME, "some.keytab"); - Application application = new Application().configuration(new org.apache - .slider.api.resource.Configuration().properties(compOps)); - - SecurityConfiguration securityConfiguration = - new SecurityConfiguration(config, application, "testCluster"); - } - - //@Test - public void testMissingPrincipalButLoginWithLocalConfig() throws Throwable { - Configuration config = new Configuration(); - config.set(CommonConfigurationKeysPublic - .HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - Map compOps = new HashMap<>(); - compOps.put(SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH, - "/some/local/path"); - Application application = new Application().configuration(new org.apache - .slider.api.resource.Configuration().properties(compOps)); - - SecurityConfiguration securityConfiguration = - new SecurityConfiguration(config, application, "testCluster"); - } - - //@Test - public void testKeypathLocationOnceLocalized() throws Throwable { - Configuration config = new Configuration(); - config.set(CommonConfigurationKeysPublic - .HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - Map compOps = new HashMap<>(); - compOps.put(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME, "some.keytab"); - Application application = new Application().configuration(new org.apache - .slider.api.resource.Configuration().properties(compOps)); - - SecurityConfiguration securityConfiguration = - new SecurityConfiguration(config, application, "testCluster"); - - assertEquals(new File(SliderKeys.KEYTAB_DIR, "some.keytab") - .getAbsolutePath(), - securityConfiguration.getKeytabFile().getAbsolutePath()); - } - - //@Test - public void testAMKeytabProvided() throws Throwable { - Configuration config = new Configuration(); - Map compOps = new HashMap<>(); - compOps.put(SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH, " "); - Application application = new Application().configuration(new org.apache - .slider.api.resource.Configuration().properties(compOps)); - - SecurityConfiguration securityConfiguration = - new SecurityConfiguration(config, application, "testCluster"); - assertFalse(securityConfiguration.isKeytabProvided()); - - compOps.put(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME, ""); - assertFalse(securityConfiguration.isKeytabProvided()); - - compOps.put(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME, "some.keytab"); - assertTrue(securityConfiguration.isKeytabProvided()); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/registry/PathEntryMarshalling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/registry/PathEntryMarshalling.java deleted file mode 100644 index b887f28..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/registry/PathEntryMarshalling.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest.registry; - -import org.apache.hadoop.registry.client.binding.JsonSerDeser; - -class PathEntryMarshalling - extends JsonSerDeser { - public PathEntryMarshalling() { - super(PathEntryResource.class); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/registry/TestRegistryRestMarshalling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/registry/TestRegistryRestMarshalling.java deleted file mode 100644 index b0b0e31..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/registry/TestRegistryRestMarshalling.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.slider.server.appmaster.web.rest.registry; - -import org.apache.hadoop.registry.client.types.ServiceRecord; -import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes; -import org.junit.Test; - -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotNull; - -/** - * This test exists because Jersey appears to behave "oddly" - * when it comes to marshalling JSON, and some of the REST resources - * appear to have trouble. - * - * This test tries to isolate it - */ -public class TestRegistryRestMarshalling { - - //@Test - public void testDeser() throws Throwable { - PathEntryMarshalling pem = new PathEntryMarshalling(); - PathEntryResource unmarshalled = pem.fromResource( - "/org/apache/slider/server/appmaster/web/rest/registry/sample.json"); - - ServiceRecord serviceRecord = unmarshalled.service; - assertNotNull(serviceRecord); - assertNotNull(serviceRecord.get(YarnRegistryAttributes.YARN_ID)); - assertNotEquals("", serviceRecord.get(YarnRegistryAttributes - .YARN_PERSISTENCE)); - } - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.java deleted file mode 100644 index 43e4f39..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.slider.server.appmaster.web.view; - -import com.google.inject.AbstractModule; -import com.google.inject.Guice; -import com.google.inject.Injector; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.state.ProviderAppState; -import org.apache.slider.server.appmaster.web.WebAppApi; -import org.apache.slider.server.appmaster.web.WebAppApiImpl; -import org.junit.Before; -import org.junit.Test; - -import java.io.PrintWriter; -import java.io.StringWriter; - -/** - * Test cluster specification block. - */ -public class TestClusterSpecificationBlock extends BaseMockAppStateTest { - - private ClusterSpecificationBlock clusterSpecBlock; - - @Before - public void setup() throws Exception { - super.setup(); - ProviderAppState providerAppState = new ProviderAppState( - "undefined", - appState); - - WebAppApiImpl inst = new WebAppApiImpl( - providerAppState, - null, - null, null); - - Injector injector = Guice.createInjector(new AbstractModule() { - @Override - protected void configure() { - bind(WebAppApi.class).toInstance(inst); - } - }); - - clusterSpecBlock = injector.getInstance(ClusterSpecificationBlock.class); - } - - //@Test - public void testJsonGeneration() { - StringWriter sw = new StringWriter(64); - PrintWriter pw = new PrintWriter(sw); - - Hamlet hamlet = new Hamlet(pw, 0, false); - - int level = hamlet.nestLevel(); - clusterSpecBlock.doRender(hamlet); - - assertEquals(level, hamlet.nestLevel()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.java deleted file mode 100644 index 56f209c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.java +++ /dev/null @@ -1,251 +0,0 @@ -/* - * 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.slider.server.appmaster.web.view; - -import com.google.inject.AbstractModule; -import com.google.inject.Guice; -import com.google.inject.Injector; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR; -import org.apache.hadoop.yarn.webapp.hamlet.HamletImpl.EImp; -import org.apache.slider.api.ClusterNode; -import org.apache.slider.server.appmaster.model.mock.BaseMockAppStateTest; -import org.apache.slider.server.appmaster.model.mock.MockContainer; -import org.apache.slider.server.appmaster.model.mock.MockContainerId; -import org.apache.slider.server.appmaster.model.mock.MockNodeId; -import org.apache.slider.server.appmaster.model.mock.MockResource; -import org.apache.slider.server.appmaster.state.ProviderAppState; -import org.apache.slider.server.appmaster.state.RoleInstance; -import org.apache.slider.server.appmaster.web.WebAppApi; -import org.apache.slider.server.appmaster.web.WebAppApiImpl; -import org.apache.slider.server.appmaster.web.view.ContainerStatsBlock.ClusterNodeNameComparator; -import org.apache.slider.server.appmaster.web.view.ContainerStatsBlock.TableAnchorContent; -import org.apache.slider.server.appmaster.web.view.ContainerStatsBlock.TableContent; -import org.junit.Before; -import org.junit.Test; - -import java.io.PrintWriter; -import java.io.StringWriter; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * Test container stats block. - */ -public class TestContainerStatsBlock extends BaseMockAppStateTest { - - private ContainerStatsBlock statsBlock; - - private Container cont1, cont2; - - @Before - public void setup() throws Exception { - super.setup(); - ProviderAppState providerAppState = new ProviderAppState( - "undefined", - appState); - - WebAppApiImpl inst = new WebAppApiImpl( - providerAppState, - null, - METRICS, null); - - Injector injector = Guice.createInjector(new WebappModule(inst)); - statsBlock = injector.getInstance(ContainerStatsBlock.class); - - cont1 = new MockContainer(); - - cont1.setId(mockContainerId(0)); - cont1.setNodeId(new MockNodeId()); - cont1.setPriority(Priority.newInstance(1)); - cont1.setResource(new MockResource(0, 0)); - - cont2 = new MockContainer(); - cont2.setId(mockContainerId(1)); - cont2.setNodeId(new MockNodeId()); - cont2.setPriority(Priority.newInstance(1)); - cont2.setResource(new MockResource(0, 0)); - } - - private static class WebappModule extends AbstractModule { - private final WebAppApiImpl instance; - - WebappModule(WebAppApiImpl instance) { - this.instance = instance; - } - - @Override - protected void configure() { - bind(WebAppApi.class).toInstance(instance); - } - } - - - public MockContainerId mockContainerId(int count) { - return new MockContainerId(applicationAttemptId, count); - } - - //@Test - public void testGetContainerInstances() { - List roles = Arrays.asList( - new RoleInstance(cont1), - new RoleInstance(cont2) - ); - Map map = statsBlock.getContainerInstances(roles); - - assertEquals(2, map.size()); - - assertTrue(map.containsKey("mockcontainer_0")); - assertEquals(map.get("mockcontainer_0"), roles.get(0)); - - assertTrue(map.containsKey("mockcontainer_1")); - assertEquals(map.get("mockcontainer_1"), roles.get(1)); - } - - //@Test - public void testGenerateRoleDetailsWithTwoColumns() { - StringWriter sw = new StringWriter(64); - PrintWriter pw = new PrintWriter(sw); - - Hamlet hamlet = new Hamlet(pw, 0, false); - - // Make a div to put the content into - DIV div = hamlet.div(); - - String detailsName = "testing"; - String selector = "selector"; - Map data = new HashMap<>(); - data.put(new ContainerStatsBlock.TableContent("Foo"), "bar"); - - int levelPrior = hamlet.nestLevel(); - statsBlock.generateRoleDetails(div, selector, detailsName, data.entrySet()); - - // Close out the div we made - // DIV._() will actually invoke the wrong method (creating

), - // explicit cast to make sure we're closing out the

- ((EImp) div)._(); - - assertEquals(levelPrior, hamlet.nestLevel()); - } - - //@Test - public void testGenerateRoleDetailsWithOneColumn() { - StringWriter sw = new StringWriter(64); - PrintWriter pw = new PrintWriter(sw); - - Hamlet hamlet = new Hamlet(pw, 0, false); - DIV div = hamlet.div(); - - String detailsName = "testing"; - String selector = "selector"; - Map data = new HashMap<>(); - data.put(new ContainerStatsBlock.TableContent("Bar"), null); - - int levelPrior = hamlet.nestLevel(); - statsBlock.generateRoleDetails(div, selector, detailsName, data.entrySet()); - - // Close out the div we made - // DIV._() will actually invoke the wrong method (creating

), - // explicit cast to make sure we're closing out the

- ((EImp) div)._(); - - assertEquals(levelPrior, hamlet.nestLevel()); - } - - //@Test - public void testGenerateRoleDetailsWithNoData() { - StringWriter sw = new StringWriter(64); - PrintWriter pw = new PrintWriter(sw); - - Hamlet hamlet = new Hamlet(pw, 0, false); - DIV div = hamlet.div(); - - String detailsName = "testing"; - String selector = "selector"; - Map data = new HashMap<>(); - - int levelPrior = hamlet.nestLevel(); - statsBlock.generateRoleDetails(div, selector, detailsName, data.entrySet()); - - // Close out the div we made - // DIV._() will actually invoke the wrong method (creating

), - // explicit cast to make sure we're closing out the

- ((EImp) div)._(); - - assertEquals(levelPrior, hamlet.nestLevel()); - } - - //@Test - public void testClusterNodeNameComparator() { - ClusterNode n1 = new ClusterNode(mockContainerId(1)), - n2 = new ClusterNode(mockContainerId(2)), - n3 = new ClusterNode(mockContainerId(3)); - - List nodes = new ArrayList(); - nodes.add(n2); - nodes.add(n3); - nodes.add(n1); - - Collections.sort(nodes, new ClusterNodeNameComparator()); - - String prevName = ""; - for (ClusterNode node : nodes) { - assertTrue(prevName.compareTo(node.name) <= 0); - prevName = node.name; - } - } - - //@Test - public void testTableContent() { - StringWriter sw = new StringWriter(64); - PrintWriter pw = new PrintWriter(sw); - TableContent tc = new TableContent("foo"); - - Hamlet hamlet = new Hamlet(pw, 0, false); - TR> tr = hamlet.table().tr(); - - int prevLevel = hamlet.nestLevel(); - // printCell should not end the tr - tc.printCell(tr); - tr._(); - assertEquals(prevLevel, hamlet.nestLevel()); - } - - //@Test - public void testTableAnchorContent() { - StringWriter sw = new StringWriter(64); - PrintWriter pw = new PrintWriter(sw); - TableContent tc = new TableAnchorContent("foo", "http://bar.com"); - - Hamlet hamlet = new Hamlet(pw, 0, false); - TR> tr = hamlet.table().tr(); - - int prevLevel = hamlet.nestLevel(); - // printCell should not end the tr - tc.printCell(tr); - tr._(); - assertEquals(prevLevel, hamlet.nestLevel()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/view/TestIndexBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/view/TestIndexBlock.java deleted file mode 100644 index eecf213..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/view/TestIndexBlock.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * 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.slider.server.appmaster.web.view; - -import com.google.inject.AbstractModule; -import com.google.inject.Guice; -import com.google.inject.Injector; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; -import org.apache.slider.server.appmaster.model.appstate.BaseMockAppStateAATest; -import org.apache.slider.server.appmaster.model.mock.MockContainer; -import org.apache.slider.server.appmaster.model.mock.MockContainerId; -import org.apache.slider.server.appmaster.model.mock.MockNodeId; -import org.apache.slider.server.appmaster.model.mock.MockResource; -import org.apache.slider.server.appmaster.state.ContainerOutcome; -import org.apache.slider.server.appmaster.state.OutstandingRequest; -import org.apache.slider.server.appmaster.state.ProviderAppState; -import org.apache.slider.server.appmaster.state.RoleStatus; -import org.apache.slider.server.appmaster.web.WebAppApi; -import org.apache.slider.server.appmaster.web.WebAppApiImpl; -import org.junit.Before; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.PrintWriter; -import java.io.StringWriter; - -/** - * Test index block. - */ -public class TestIndexBlock extends BaseMockAppStateAATest { - private static final Logger LOG = - LoggerFactory.getLogger(TestIndexBlock.class); - - private IndexBlock indexBlock; - - private Container cont1, cont2; - - @Before - public void setup() throws Exception { - super.setup(); - assertNotNull(appState); - ProviderAppState providerAppState = new ProviderAppState( - "undefined", - appState); - - WebAppApiImpl inst = new WebAppApiImpl( - providerAppState, - null, - METRICS, null); - - Injector injector = Guice.createInjector(new AbstractModule() { - @Override - protected void configure() { - bind(WebAppApi.class).toInstance(inst); - } - }); - - indexBlock = injector.getInstance(IndexBlock.class); - - cont1 = new MockContainer(); - cont1.setId(new MockContainerId(applicationAttemptId, 0)); - cont1.setNodeId(new MockNodeId()); - cont1.setPriority(Priority.newInstance(1)); - cont1.setResource(new MockResource(0, 0)); - - cont2 = new MockContainer(); - cont2.setId(new MockContainerId(applicationAttemptId, 1)); - cont2.setNodeId(new MockNodeId()); - cont2.setPriority(Priority.newInstance(1)); - cont2.setResource(new MockResource(0, 0)); - } - - //@Test - public void testIndex() { - RoleStatus role0 = getRole0Status(); - RoleStatus role1 = getRole1Status(); - RoleStatus role2 = getRole2Status(); - - int role0Desired = 8; - - role0.setDesired(role0Desired); - int role0Actual = 5; - int role0Requested = role0Desired - role0Actual; - for (int i = 0; i < role0Actual; i++) { - appState.incRunningContainers(role0); - } - assertEquals(role0.getRunning(), role0Actual); - for (int i = 0; i < role0Requested; i++) { - appState.incRequestedContainers(role0); - } - assertEquals(role0.getRequested(), role0Requested); - - int role0Failures = 2; - - appState.incFailedContainers(role0, ContainerOutcome.Failed); - appState.incFailedContainers(role0, ContainerOutcome.Failed); - - RoleStatus aaRole = getAaRole(); - // all aa roles fields are in the - int aaroleDesired = 200; - aaRole.setDesired(aaroleDesired); - int aaroleActual = 90; - int aaroleActive = 1; - int aaroleRequested = aaroleDesired - aaroleActual; - int aarolePending = aaroleRequested - 1; - int aaroleFailures = 0; - for (int i = 0; i < aaroleActual; i++) { - appState.incRunningContainers(aaRole); - } - assertEquals(aaRole.getRunning(), aaroleActual); - aaRole.setOutstandingAArequest(new OutstandingRequest(2, "")); - // add a requested - appState.incRequestedContainers(aaRole); - aaRole.getComponentMetrics().pendingAAContainers.set(aarolePending); - assertEquals(aaRole.getAAPending(), aarolePending); - - assertEquals(aaRole.getActualAndRequested(), aaroleActual + 1); - StringWriter sw = new StringWriter(64); - PrintWriter pw = new PrintWriter(sw); - - Hamlet hamlet = new Hamlet(pw, 0, false); - - indexBlock.doIndex(hamlet, "accumulo"); - - String body = sw.toString(); - LOG.info(body); - // verify role data came out - assertTrue(body.contains("role0")); - assertContains(role0Desired, body); - assertContains(role0Actual, body); - assertContains(role0Requested, body); - assertContains(role0Failures, body); - - assertTrue(body.contains("role1")); - assertTrue(body.contains("role2")); - - assertContains(aaroleDesired, body); - assertContains(aaroleActual, body); -// assertContains(aaroleRequested, body) - assertContains(aaroleFailures, body); - assertTrue(body.contains(indexBlock.buildAADetails(true, aarolePending))); - - // verify that the sorting took place - assertTrue(body.indexOf("role0") < body.indexOf("role1")); - assertTrue(body.indexOf("role1") < body.indexOf("role2")); - - assertFalse(body.contains(IndexBlock.ALL_CONTAINERS_ALLOCATED)); - // role - } - - void assertContains(int ex, String html) { - assertStringContains(Integer.toString(ex), html); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/management/TestGauges.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/management/TestGauges.java deleted file mode 100644 index 11ebabe..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/management/TestGauges.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.slider.server.management; - -import org.apache.slider.server.appmaster.management.LongGauge; -import org.apache.slider.utils.SliderTestBase; -import org.junit.Test; - -/** - * Test gauges. - */ -public class TestGauges extends SliderTestBase { - - //@Test - public void testLongGaugeOperations() throws Throwable { - LongGauge gauge = new LongGauge(); - assertEquals(0, gauge.get()); - gauge.inc(); - assertEquals(1, gauge.get()); - gauge.inc(); - assertEquals(2, gauge.get()); - gauge.inc(); - assertEquals(3, gauge.get()); - assertEquals(gauge.getValue().longValue(), gauge.get()); - assertEquals(gauge.getCount().longValue(), gauge.get()); - - gauge.dec(); - assertEquals(2, gauge.get()); - assertEquals(1, gauge.decToFloor(1)); - assertEquals(1, gauge.get()); - assertEquals(0, gauge.decToFloor(1)); - assertEquals(0, gauge.decToFloor(1)); - assertEquals(0, gauge.decToFloor(0)); - - gauge.set(4); - assertEquals(0, gauge.decToFloor(8)); - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/MockService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/MockService.java deleted file mode 100644 index 588f621..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/MockService.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.service.ServiceStateException; - -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -public class MockService extends AbstractService { - private final boolean fail; - private final int lifespan; - private final ExecutorService executorService = - Executors.newSingleThreadExecutor(); - - MockService() { - this("mock", false, -1); - } - - MockService(String name, boolean fail, int lifespan) { - super(name); - this.fail = fail; - this.lifespan = lifespan; - } - - @Override - protected void serviceStart() throws Exception { - //act on the lifespan here - if (lifespan > 0) { - executorService.submit(new Runnable() { - @Override - public void run() { - try { - Thread.sleep(lifespan); - } catch (InterruptedException ignored) { - - } - finish(); - } - }); - } else { - if (lifespan == 0) { - finish(); - } else { - //continue until told not to - } - } - } - - void finish() { - if (fail) { - ServiceStateException e = - new ServiceStateException(getName() + " failed"); - - noteFailure(e); - stop(); - throw e; - } else { - stop(); - } - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ParentWorkflowTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ParentWorkflowTestBase.java deleted file mode 100644 index a11a1cf..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ParentWorkflowTestBase.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import org.apache.hadoop.service.Service; - -/** - * Extends {@link WorkflowServiceTestBase} with parent-specific operations - * and logic to build up and run the parent service - */ -public abstract class ParentWorkflowTestBase extends WorkflowServiceTestBase { - - /** - * Wait a second for the service parent to stop - * @param parent the service to wait for - */ - protected void waitForParentToStop(ServiceParent parent) { - waitForParentToStop(parent, 1000); - } - - /** - * Wait for the service parent to stop - * @param parent the service to wait for - * @param timeout time in milliseconds - */ - protected void waitForParentToStop(ServiceParent parent, int timeout) { - boolean stop = parent.waitForServiceToStop(timeout); - if (!stop) { - logState(parent); - fail("Service failed to stop : after " + timeout + " millis " + parent); - } - } - - /** - * Subclasses are require to implement this and return an instance of a - * ServiceParent - * @param services a possibly empty list of services - * @return an inited -but -not-started- service parent instance - */ - protected abstract ServiceParent buildService(Service... services); - - /** - * Use {@link #buildService(Service...)} to create service and then start it - * @param services - * @return - */ - protected ServiceParent startService(Service... services) { - ServiceParent parent = buildService(services); - //expect service to start and stay started - parent.start(); - return parent; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ProcessCommandFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ProcessCommandFactory.java deleted file mode 100644 index 4a19417..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ProcessCommandFactory.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import org.apache.hadoop.util.Shell; - -import java.io.File; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -/** - * A source of commands, with the goal being to allow for adding different - * implementations for different platforms - */ -public class ProcessCommandFactory { - - protected ProcessCommandFactory() { - } - - /** - * The command to list a directory - * @param dir directory - * @return commands - */ - public List ls(File dir) { - List commands; - if (!Shell.WINDOWS) { - commands = Arrays.asList("ls","-1", dir.getAbsolutePath()); - } else { - commands = Arrays.asList("cmd", "/c", "dir", dir.getAbsolutePath()); - } - return commands; - } - - /** - * Echo some text to stdout - * @param text text - * @return commands - */ - public List echo(String text) { - List commands = new ArrayList(5); - commands.add("echo"); - commands.add(text); - return commands; - } - - /** - * print env variables - * @return commands - */ - public List env() { - List commands; - if (!Shell.WINDOWS) { - commands = Arrays.asList("env"); - } else { - commands = Arrays.asList("cmd", "/c", "set"); - } - return commands; - } - - /** - * execute a command that returns with an error code that will - * be converted into a number - * @return commands - */ - public List exitFalse() { - List commands = new ArrayList(2); - commands.add("false"); - return commands; - } - - /** - * Create a process command factory for this OS - * @return - */ - public static ProcessCommandFactory createProcessCommandFactory() { - return new ProcessCommandFactory(); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/SimpleRunnable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/SimpleRunnable.java deleted file mode 100644 index 1f330f4..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/SimpleRunnable.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -/** - * Test runnable that can be made to exit, or throw an exception - * during its run - */ -class SimpleRunnable implements Runnable { - boolean throwException = false; - - - SimpleRunnable() { - } - - SimpleRunnable(boolean throwException) { - this.throwException = throwException; - } - - @Override - public synchronized void run() { - try { - if (throwException) { - throw new RuntimeException("SimpleRunnable"); - } - } finally { - this.notify(); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowClosingService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowClosingService.java deleted file mode 100644 index 19f40e9..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowClosingService.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import org.apache.hadoop.conf.Configuration; -import org.junit.Test; - -import java.io.Closeable; -import java.io.IOException; - -public class TestWorkflowClosingService extends WorkflowServiceTestBase { - - //@Test - public void testSimpleClose() throws Throwable { - ClosingService svc = instance(false); - OpenClose openClose = svc.getCloseable(); - assertFalse(openClose.closed); - svc.stop(); - assertTrue(openClose.closed); - } - - //@Test - public void testNullClose() throws Throwable { - ClosingService svc = new ClosingService("", null); - svc.init(new Configuration()); - svc.start(); - assertNull(svc.getCloseable()); - svc.stop(); - } - - //@Test - public void testFailingClose() throws Throwable { - ClosingService svc = instance(false); - OpenClose openClose = svc.getCloseable(); - openClose.raiseExceptionOnClose = true; - svc.stop(); - assertTrue(openClose.closed); - Throwable cause = svc.getFailureCause(); - assertNotNull(cause); - - //retry should be a no-op - svc.close(); - } - - //@Test - public void testDoubleClose() throws Throwable { - ClosingService svc = instance(false); - OpenClose openClose = svc.getCloseable(); - openClose.raiseExceptionOnClose = true; - svc.stop(); - assertTrue(openClose.closed); - Throwable cause = svc.getFailureCause(); - assertNotNull(cause); - openClose.closed = false; - svc.stop(); - assertEquals(cause, svc.getFailureCause()); - } - - /** - * This does not recurse forever, as the service has already entered the - * STOPPED state before the inner close tries to stop it -that operation - * is a no-op - * @throws Throwable - */ - //@Test - public void testCloseSelf() throws Throwable { - ClosingService svc = - new ClosingService(""); - svc.setCloseable(svc); - svc.stop(); - } - - - private ClosingService instance(boolean raiseExceptionOnClose) { - ClosingService svc = new ClosingService(new OpenClose( - raiseExceptionOnClose)); - svc.init(new Configuration()); - svc.start(); - return svc; - } - - private static class OpenClose implements Closeable { - public boolean closed = false; - public boolean raiseExceptionOnClose; - - private OpenClose(boolean raiseExceptionOnClose) { - this.raiseExceptionOnClose = raiseExceptionOnClose; - } - - @Override - public void close() throws IOException { - if (!closed) { - closed = true; - if (raiseExceptionOnClose) { - throw new IOException("OpenClose"); - } - } - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowCompositeService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowCompositeService.java deleted file mode 100644 index 0cd1ac9..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowCompositeService.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * 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.slider.server.services.workflow; - - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.Service; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TestWorkflowCompositeService extends ParentWorkflowTestBase { - private static final Logger - log = LoggerFactory.getLogger(TestWorkflowCompositeService.class); - - //@Test - public void testSingleChild() throws Throwable { - Service parent = startService(new MockService()); - parent.stop(); - } - - //@Test - public void testSingleChildTerminating() throws Throwable { - ServiceParent parent = - startService(new MockService("1", false, 100)); - waitForParentToStop(parent); - } - - //@Test - public void testSingleChildFailing() throws Throwable { - ServiceParent parent = - startService(new MockService("1", true, 100)); - waitForParentToStop(parent); - assert parent.getFailureCause() != null; - } - - //@Test - public void testTwoChildren() throws Throwable { - MockService one = new MockService("one", false, 100); - MockService two = new MockService("two", false, 100); - ServiceParent parent = startService(one, two); - waitForParentToStop(parent); - assertStopped(one); - assertStopped(two); - } - - //@Test - public void testCallableChild() throws Throwable { - - MockService one = new MockService("one", false, 100); - CallableHandler handler = new CallableHandler("hello"); - WorkflowCallbackService ens = - new WorkflowCallbackService("handler", handler, 100, true); - MockService two = new MockService("two", false, 100); - ServiceParent parent = startService(one, ens, two); - waitForParentToStop(parent); - assertStopped(one); - assertStopped(ens); - assertStopped(two); - assertTrue(handler.notified); - String s = ens.getScheduledFuture().get(); - assertEquals("hello", s); - } - - //@Test - public void testNestedComposite() throws Throwable { - MockService one = new MockService("one", false, 100); - MockService two = new MockService("two", false, 100); - ServiceParent parent = buildService(one, two); - ServiceParent outer = startService(parent); - assertTrue(outer.waitForServiceToStop(1000)); - assertStopped(one); - assertStopped(two); - } - - //@Test - public void testFailingComposite() throws Throwable { - MockService one = new MockService("one", true, 10); - MockService two = new MockService("two", false, 1000); - ServiceParent parent = startService(one, two); - waitForParentToStop(parent); - assertStopped(one); - assertStopped(two); - assertNotNull(one.getFailureCause()); - assertNotNull(parent.getFailureCause()); - assertEquals(one.getFailureCause(), parent.getFailureCause()); - } - - @Override - public ServiceParent buildService(Service... services) { - ServiceParent parent = - new WorkflowCompositeService("test", services); - parent.init(new Configuration()); - return parent; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowExecutorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowExecutorService.java deleted file mode 100644 index 38cc886..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowExecutorService.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import org.junit.Test; - -import java.util.concurrent.ExecutorService; - - -/** - * Basic tests for executor service - */ -public class TestWorkflowExecutorService extends WorkflowServiceTestBase { - - //@Test - public void testAsyncRun() throws Throwable { - - ExecutorSvc svc = run(new ExecutorSvc()); - ServiceTerminatingRunnable runnable = new ServiceTerminatingRunnable(svc, - new SimpleRunnable()); - - // synchronous in-thread execution - svc.execute(runnable); - Thread.sleep(1000); - assertStopped(svc); - } - - //@Test - public void testFailureRun() throws Throwable { - - ExecutorSvc svc = run(new ExecutorSvc()); - ServiceTerminatingRunnable runnable = new ServiceTerminatingRunnable(svc, - new SimpleRunnable(true)); - - // synchronous in-thread execution - svc.execute(runnable); - Thread.sleep(1000); - assertStopped(svc); - assertNotNull(runnable.getException()); - } - - private static class ExecutorSvc - extends WorkflowExecutorService { - private ExecutorSvc() { - super("ExecutorService", - ServiceThreadFactory.singleThreadExecutor("test", true)); - } - - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowRpcService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowRpcService.java deleted file mode 100644 index 758c64f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowRpcService.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.ipc.Server; -import org.junit.Test; - -import java.io.IOException; -import java.net.InetSocketAddress; - -public class TestWorkflowRpcService extends WorkflowServiceTestBase { - - //@Test - public void testCreateMockRPCService() throws Throwable { - MockRPC rpc = new MockRPC(); - rpc.start(); - assertTrue(rpc.started); - rpc.getListenerAddress(); - rpc.stop(); - assertTrue(rpc.stopped); - } - - //@Test - public void testLifecycle() throws Throwable { - MockRPC rpc = new MockRPC(); - WorkflowRpcService svc = new WorkflowRpcService("test", rpc); - run(svc); - assertTrue(rpc.started); - svc.getConnectAddress(); - svc.stop(); - assertTrue(rpc.stopped); - } - - //@Test - public void testStartFailure() throws Throwable { - MockRPC rpc = new MockRPC(); - rpc.failOnStart = true; - WorkflowRpcService svc = new WorkflowRpcService("test", rpc); - svc.init(new Configuration()); - try { - svc.start(); - fail("expected an exception"); - } catch (RuntimeException e) { - assertEquals("failOnStart", e.getMessage()); - } - svc.stop(); - assertTrue(rpc.stopped); - } - - private static class MockRPC extends Server { - - public boolean stopped; - public boolean started; - public boolean failOnStart; - - private MockRPC() throws IOException { - super("localhost", 0, null, 1, new Configuration()); - } - - @Override - public synchronized void start() { - if (failOnStart) { - throw new RuntimeException("failOnStart"); - } - started = true; - super.start(); - } - - @Override - public synchronized void stop() { - stopped = true; - super.stop(); - } - - @Override - public synchronized InetSocketAddress getListenerAddress() { - return super.getListenerAddress(); - } - - @Override - public Writable call(RPC.RpcKind rpcKind, - String protocol, - Writable param, - long receiveTime) throws Exception { - return null; - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowSequenceService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowSequenceService.java deleted file mode 100644 index b683641..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowSequenceService.java +++ /dev/null @@ -1,151 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.Service; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TestWorkflowSequenceService extends ParentWorkflowTestBase { - private static final Logger - log = LoggerFactory.getLogger(TestWorkflowSequenceService.class); - - //@Test - public void testSingleSequence() throws Throwable { - ServiceParent parent = startService(new MockService()); - parent.stop(); - } - - //@Test - public void testEmptySequence() throws Throwable { - ServiceParent parent = startService(); - waitForParentToStop(parent); - } - - //@Test - public void testSequence() throws Throwable { - MockService one = new MockService("one", false, 100); - MockService two = new MockService("two", false, 100); - ServiceParent parent = startService(one, two); - waitForParentToStop(parent); - assertStopped(one); - assertStopped(two); - assert ((WorkflowSequenceService) parent).getPreviousService().equals(two); - } - - //@Test - public void testCallableChild() throws Throwable { - - MockService one = new MockService("one", false, 100); - CallableHandler handler = new CallableHandler("hello"); - WorkflowCallbackService ens = - new WorkflowCallbackService("handler", handler, 100, true); - MockService two = new MockService("two", false, 100); - ServiceParent parent = startService(one, ens, two); - waitForParentToStop(parent); - assertStopped(one); - assertStopped(ens); - assertStopped(two); - assertTrue(handler.notified); - String s = ens.getScheduledFuture().get(); - assertEquals("hello", s); - } - - - //@Test - public void testFailingSequence() throws Throwable { - MockService one = new MockService("one", true, 100); - MockService two = new MockService("two", false, 100); - WorkflowSequenceService parent = - (WorkflowSequenceService) startService(one, two); - waitForParentToStop(parent); - assertStopped(one); - assertInState(two, Service.STATE.NOTINITED); - assertEquals(one, parent.getPreviousService()); - } - - - //@Test - public void testFailInStartNext() throws Throwable { - MockService one = new MockService("one", false, 100); - MockService two = new MockService("two", true, 0); - MockService three = new MockService("3", false, 0); - ServiceParent parent = startService(one, two, three); - waitForParentToStop(parent); - assertStopped(one); - assertStopped(two); - Throwable failureCause = two.getFailureCause(); - assertNotNull(failureCause); - Throwable parentFailureCause = parent.getFailureCause(); - assertNotNull(parentFailureCause); - assertEquals(parentFailureCause, failureCause); - assertInState(three, Service.STATE.NOTINITED); - } - - //@Test - public void testSequenceInSequence() throws Throwable { - MockService one = new MockService("one", false, 100); - MockService two = new MockService("two", false, 100); - ServiceParent parent = buildService(one, two); - ServiceParent outer = startService(parent); - waitForParentToStop(parent); - assertStopped(one); - assertStopped(two); - } - - //@Test - public void testVarargsConstructor() throws Throwable { - MockService one = new MockService("one", false, 100); - MockService two = new MockService("two", false, 100); - ServiceParent parent = new WorkflowSequenceService("test", one, two); - parent.init(new Configuration()); - parent.start(); - waitForParentToStop(parent); - assertStopped(one); - assertStopped(two); - } - - - //@Test - public void testAddChild() throws Throwable { - MockService one = new MockService("one", false, 5000); - MockService two = new MockService("two", false, 100); - ServiceParent parent = startService(one, two); - CallableHandler handler = new CallableHandler("hello"); - WorkflowCallbackService ens = - new WorkflowCallbackService("handler", handler, 100, true); - parent.addService(ens); - waitForParentToStop(parent, 10000); - assertStopped(one); - assertStopped(two); - assertStopped(ens); - assertStopped(two); - assertEquals("hello", ens.getScheduledFuture().get()); - } - - public WorkflowSequenceService buildService(Service... services) { - WorkflowSequenceService parent = - new WorkflowSequenceService("test", services); - parent.init(new Configuration()); - return parent; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowServiceTerminatingRunnable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowServiceTerminatingRunnable.java deleted file mode 100644 index a667432..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowServiceTerminatingRunnable.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import org.junit.Test; - - -public class TestWorkflowServiceTerminatingRunnable extends WorkflowServiceTestBase { - - //@Test - public void testNoservice() throws Throwable { - - try { - new ServiceTerminatingRunnable(null, new SimpleRunnable()); - fail("unexpected "); - } catch (IllegalArgumentException e) { - - // expected - } - } - - - //@Test - public void testBasicRun() throws Throwable { - - WorkflowCompositeService svc = run(new WorkflowCompositeService()); - ServiceTerminatingRunnable runnable = new ServiceTerminatingRunnable(svc, - new SimpleRunnable()); - - // synchronous in-thread execution - runnable.run(); - assertStopped(svc); - } - - //@Test - public void testFailureRun() throws Throwable { - - WorkflowCompositeService svc = run(new WorkflowCompositeService()); - ServiceTerminatingRunnable runnable = - new ServiceTerminatingRunnable(svc, new SimpleRunnable(true)); - - // synchronous in-thread execution - runnable.run(); - assertStopped(svc); - assertNotNull(runnable.getException()); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/WorkflowServiceTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/WorkflowServiceTestBase.java deleted file mode 100644 index f38bd9d..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/WorkflowServiceTestBase.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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.slider.server.services.workflow; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.Service; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.rules.TestName; -import org.junit.rules.Timeout; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; -import java.util.Locale; -import java.util.concurrent.Callable; - -/** - * Test base for workflow service tests. - */ -public abstract class WorkflowServiceTestBase extends Assert { - private static final Logger - log = LoggerFactory.getLogger(WorkflowServiceTestBase.class); - - /** - * Set the timeout for every test - */ - @Rule - public Timeout testTimeout = new Timeout(15000); - - @Rule - public TestName name = new TestName(); - - @Before - public void nameThread() { - Thread.currentThread().setName("JUnit"); - } - - - protected void assertInState(Service service, Service.STATE expected) { - Service.STATE actual = service.getServiceState(); - if (actual != expected) { - fail("Service " + service.getName() + " in state " + actual - + " -expected " + expected); - } - } - - protected void assertStopped(Service service) { - assertInState(service, Service.STATE.STOPPED); - } - - protected void logState(ServiceParent p) { - logService(p); - for (Service s : p.getServices()) { - logService(s); - } - } - - protected void logService(Service s) { - log.info(s.toString()); - Throwable failureCause = s.getFailureCause(); - if (failureCause != null) { - log.info("Failed in state {} with {}", s.getFailureState(), - failureCause); - } - } - - /** - * Init and start a service - * @param svc the service - * @return the service - */ - protected S run(S svc) { - svc.init(new Configuration()); - svc.start(); - return svc; - } - - /** - * Handler for callable events - */ - public static class CallableHandler implements Callable { - public volatile boolean notified = false; - public final String result; - - public CallableHandler(String result) { - this.result = result; - } - - @Override - public String call() throws Exception { - log.info("CallableHandler::call"); - notified = true; - return result; - } - } - - /** - * Assert that a string is in an output list. Fails fast if the output - * list is empty - * @param text text to scan for - * @param output list of output lines. - */ - public void assertStringInOutput(String text, List output) { - assertTrue("Empty output list", !output.isEmpty()); - boolean found = false; - StringBuilder builder = new StringBuilder(); - for (String s : output) { - builder.append(s.toLowerCase(Locale.ENGLISH)).append('\n'); - if (s.contains(text)) { - found = true; - break; - } - } - - if (!found) { - String message = - "Text \"" + text + "\" not found in " + output.size() + " lines\n"; - fail(message + builder.toString()); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/ContractTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/ContractTestUtils.java deleted file mode 100644 index fc51e31..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/ContractTestUtils.java +++ /dev/null @@ -1,901 +0,0 @@ -/* - * 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.slider.utils; - -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.junit.Assert; -import org.junit.internal.AssumptionViolatedException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.EOFException; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Arrays; -import java.util.Properties; -import java.util.UUID; - -/** - * Utilities used across test cases to make assertions about filesystems - * -assertions which fail with useful information. - * This is lifted from Hadoop common Test; that JAR isn't published, so - * we have to make do. - */ -public class ContractTestUtils extends Assert { - - private static final Logger LOG = - LoggerFactory.getLogger(ContractTestUtils.class); - - public static final String IO_FILE_BUFFER_SIZE = "io.file.buffer.size"; - - // For scale testing, we can repeatedly write small chunk data to generate - // a large file. - public static final String IO_CHUNK_BUFFER_SIZE = "io.chunk.buffer.size"; - public static final int DEFAULT_IO_CHUNK_BUFFER_SIZE = 128; - public static final String IO_CHUNK_MODULUS_SIZE = "io.chunk.modulus.size"; - public static final int DEFAULT_IO_CHUNK_MODULUS_SIZE = 128; - - /** - * Assert that a property in the property set matches the expected value - * @param props property set - * @param key property name - * @param expected expected value. If null, the property must not be in the set - */ - public static void assertPropertyEquals(Properties props, - String key, - String expected) { - String val = props.getProperty(key); - if (expected == null) { - assertNull("Non null property " + key + " = " + val, val); - } else { - assertEquals("property " + key + " = " + val, - expected, - val); - } - } - - /** - * - * Write a file and read it in, validating the result. Optional flags control - * whether file overwrite operations should be enabled, and whether the - * file should be deleted afterwards. - * - * If there is a mismatch between what was written and what was expected, - * a small range of bytes either side of the first error are logged to aid - * diagnosing what problem occurred -whether it was a previous file - * or a corrupting of the current file. This assumes that two - * sequential runs to the same path use datasets with different character - * moduli. - * - * @param fs filesystem - * @param path path to write to - * @param len length of data - * @param overwrite should the create option allow overwrites? - * @param delete should the file be deleted afterwards? -with a verification - * that it worked. Deletion is not attempted if an assertion has failed - * earlier -it is not in a finally{} block. - * @throws IOException IO problems - */ - public static void writeAndRead(FileSystem fs, - Path path, - byte[] src, - int len, - int blocksize, - boolean overwrite, - boolean delete) throws IOException { - fs.mkdirs(path.getParent()); - - writeDataset(fs, path, src, len, blocksize, overwrite); - - byte[] dest = readDataset(fs, path, len); - - compareByteArrays(src, dest, len); - - if (delete) { - rejectRootOperation(path); - boolean deleted = fs.delete(path, false); - assertTrue("Deleted", deleted); - assertPathDoesNotExist(fs, "Cleanup failed", path); - } - } - - /** - * Write a file. - * Optional flags control - * whether file overwrite operations should be enabled - * @param fs filesystem - * @param path path to write to - * @param len length of data - * @param overwrite should the create option allow overwrites? - * @throws IOException IO problems - */ - public static void writeDataset(FileSystem fs, - Path path, - byte[] src, - int len, - int buffersize, - boolean overwrite) throws IOException { - assertTrue( - "Not enough data in source array to write " + len + " bytes", - src.length >= len); - FSDataOutputStream out = fs.create(path, - overwrite, - fs.getConf() - .getInt(IO_FILE_BUFFER_SIZE, - 4096), - (short) 1, - buffersize); - out.write(src, 0, len); - out.close(); - assertFileHasLength(fs, path, len); - } - - /** - * Read the file and convert to a byte dataset. - * This implements readfully internally, so that it will read - * in the file without ever having to seek() - * @param fs filesystem - * @param path path to read from - * @param len length of data to read - * @return the bytes - * @throws IOException IO problems - */ - public static byte[] readDataset(FileSystem fs, Path path, int len) - throws IOException { - FSDataInputStream in = fs.open(path); - byte[] dest = new byte[len]; - int offset =0; - int nread = 0; - try { - while (nread < len) { - int nbytes = in.read(dest, offset + nread, len - nread); - if (nbytes < 0) { - throw new EOFException("End of file reached before reading fully."); - } - nread += nbytes; - } - } finally { - in.close(); - } - return dest; - } - - /** - * Read a file, verify its length and contents match the expected array - * @param fs filesystem - * @param path path to file - * @param original original dataset - * @throws IOException IO Problems - */ - public static void verifyFileContents(FileSystem fs, - Path path, - byte[] original) throws IOException { - FileStatus stat = fs.getFileStatus(path); - String statText = stat.toString(); - assertTrue("not a file " + statText, stat.isFile()); - assertEquals("wrong length " + statText, original.length, stat.getLen()); - byte[] bytes = readDataset(fs, path, original.length); - compareByteArrays(original,bytes,original.length); - } - - /** - * Verify that the read at a specific offset in a stream - * matches that expected - * @param stm stream - * @param fileContents original file contents - * @param seekOff seek offset - * @param toRead number of bytes to read - * @throws IOException IO problems - */ - public static void verifyRead(FSDataInputStream stm, byte[] fileContents, - int seekOff, int toRead) throws IOException { - byte[] out = new byte[toRead]; - stm.seek(seekOff); - stm.readFully(out); - byte[] expected = Arrays.copyOfRange(fileContents, seekOff, - seekOff + toRead); - compareByteArrays(expected, out,toRead); - } - - /** - * Assert that tthe array original[0..len] and received[] are equal. - * A failure triggers the logging of the bytes near where the first - * difference surfaces. - * @param original source data - * @param received actual - * @param len length of bytes to compare - */ - public static void compareByteArrays(byte[] original, - byte[] received, - int len) { - assertEquals("Number of bytes read != number written", - len, received.length); - int errors = 0; - int first_error_byte = -1; - for (int i = 0; i < len; i++) { - if (original[i] != received[i]) { - if (errors == 0) { - first_error_byte = i; - } - errors++; - } - } - - if (errors > 0) { - String message = String.format(" %d errors in file of length %d", - errors, len); - LOG.warn(message); - // the range either side of the first error to print - // this is a purely arbitrary number, to aid user debugging - final int overlap = 10; - for (int i = Math.max(0, first_error_byte - overlap); - i < Math.min(first_error_byte + overlap, len); - i++) { - byte actual = received[i]; - byte expected = original[i]; - String letter = toChar(actual); - String line = String.format("[%04d] %2x %s\n", i, actual, letter); - if (expected != actual) { - line = String.format("[%04d] %2x %s -expected %2x %s\n", - i, - actual, - letter, - expected, - toChar(expected)); - } - LOG.warn(line); - } - fail(message); - } - } - - /** - * Convert a byte to a character for printing. If the - * byte value is < 32 -and hence unprintable- the byte is - * returned as a two digit hex value - * @param b byte - * @return the printable character string - */ - public static String toChar(byte b) { - if (b >= 0x20) { - return Character.toString((char) b); - } else { - return String.format("%02x", b); - } - } - - /** - * Convert a buffer to a string, character by character - * @param buffer input bytes - * @return a string conversion - */ - public static String toChar(byte[] buffer) { - StringBuilder builder = new StringBuilder(buffer.length); - for (byte b : buffer) { - builder.append(toChar(b)); - } - return builder.toString(); - } - - public static byte[] toAsciiByteArray(String s) { - char[] chars = s.toCharArray(); - int len = chars.length; - byte[] buffer = new byte[len]; - for (int i = 0; i < len; i++) { - buffer[i] = (byte) (chars[i] & 0xff); - } - return buffer; - } - - /** - * Cleanup at the end of a test run - * @param action action triggering the operation (for use in logging) - * @param fileSystem filesystem to work with. May be null - * @param cleanupPath path to delete as a string - */ - public static void cleanup(String action, - FileSystem fileSystem, - String cleanupPath) { - if (fileSystem == null) { - return; - } - Path path = new Path(cleanupPath).makeQualified(fileSystem.getUri(), - fileSystem.getWorkingDirectory()); - cleanup(action, fileSystem, path); - } - - /** - * Cleanup at the end of a test run - * @param action action triggering the operation (for use in logging) - * @param fileSystem filesystem to work with. May be null - * @param path path to delete - */ - public static void cleanup(String action, FileSystem fileSystem, Path path) { - noteAction(action); - try { - rm(fileSystem, path, true, false); - } catch (Exception e) { - LOG.error("Error deleting in "+ action + " - " + path + ": " + e, e); - } - } - - /** - * Delete a directory. There's a safety check for operations against the - * root directory -these are intercepted and rejected with an IOException - * unless the allowRootDelete flag is true - * @param fileSystem filesystem to work with. May be null - * @param path path to delete - * @param recursive flag to enable recursive delete - * @param allowRootDelete can the root directory be deleted? - * @throws IOException on any problem. - */ - public static boolean rm(FileSystem fileSystem, - Path path, - boolean recursive, - boolean allowRootDelete) throws - IOException { - if (fileSystem != null) { - rejectRootOperation(path, allowRootDelete); - if (fileSystem.exists(path)) { - return fileSystem.delete(path, recursive); - } - } - return false; - - } - - /** - * Block any operation on the root path. This is a safety check - * @param path path in the filesystem - * @param allowRootOperation can the root directory be manipulated? - * @throws IOException if the operation was rejected - */ - public static void rejectRootOperation(Path path, - boolean allowRootOperation) throws IOException { - if (path.isRoot() && !allowRootOperation) { - throw new IOException("Root directory operation rejected: " + path); - } - } - - /** - * Block any operation on the root path. This is a safety check - * @param path path in the filesystem - * @throws IOException if the operation was rejected - */ - public static void rejectRootOperation(Path path) throws IOException { - rejectRootOperation(path, false); - } - - - public static void noteAction(String action) { - if (LOG.isDebugEnabled()) { - LOG.debug("============== "+ action +" ============="); - } - } - - /** - * downgrade a failure to a message and a warning, then an - * exception for the Junit test runner to mark as failed - * @param message text message - * @param failure what failed - * @throws AssumptionViolatedException always - */ - public static void downgrade(String message, Throwable failure) { - LOG.warn("Downgrading test " + message, failure); - AssumptionViolatedException ave = - new AssumptionViolatedException(failure, null); - throw ave; - } - - /** - * report an overridden test as unsupported - * @param message message to use in the text - * @throws AssumptionViolatedException always - */ - public static void unsupported(String message) { - skip(message); - } - - /** - * report a test has been skipped for some reason - * @param message message to use in the text - * @throws AssumptionViolatedException always - */ - public static void skip(String message) { - LOG.info("Skipping: {}", message); - throw new AssumptionViolatedException(message); - } - - /** - * Fail with an exception that was received - * @param text text to use in the exception - * @param thrown a (possibly null) throwable to init the cause with - * @throws AssertionError with the text and throwable -always - */ - public static void fail(String text, Throwable thrown) { - AssertionError e = new AssertionError(text); - e.initCause(thrown); - throw e; - } - - /** - * Make an assertion about the length of a file - * @param fs filesystem - * @param path path of the file - * @param expected expected length - * @throws IOException on File IO problems - */ - public static void assertFileHasLength(FileSystem fs, Path path, - int expected) throws IOException { - FileStatus status = fs.getFileStatus(path); - assertEquals( - "Wrong file length of file " + path + " status: " + status, - expected, - status.getLen()); - } - - /** - * Assert that a path refers to a directory - * @param fs filesystem - * @param path path of the directory - * @throws IOException on File IO problems - */ - public static void assertIsDirectory(FileSystem fs, - Path path) throws IOException { - FileStatus fileStatus = fs.getFileStatus(path); - assertIsDirectory(fileStatus); - } - - /** - * Assert that a path refers to a directory - * @param fileStatus stats to check - */ - public static void assertIsDirectory(FileStatus fileStatus) { - assertTrue("Should be a directory -but isn't: " + fileStatus, - fileStatus.isDirectory()); - } - - /** - * Write the text to a file, returning the converted byte array - * for use in validating the round trip - * @param fs filesystem - * @param path path of file - * @param text text to write - * @param overwrite should the operation overwrite any existing file? - * @return the read bytes - * @throws IOException on IO problems - */ - public static byte[] writeTextFile(FileSystem fs, - Path path, - String text, - boolean overwrite) throws IOException { - byte[] bytes = new byte[0]; - if (text != null) { - bytes = toAsciiByteArray(text); - } - createFile(fs, path, overwrite, bytes); - return bytes; - } - - /** - * Create a file - * @param fs filesystem - * @param path path to write - * @param overwrite overwrite flag - * @param data source dataset. Can be null - * @throws IOException on any problem - */ - public static void createFile(FileSystem fs, - Path path, - boolean overwrite, - byte[] data) throws IOException { - FSDataOutputStream stream = fs.create(path, overwrite); - if (data != null && data.length > 0) { - stream.write(data); - } - stream.close(); - } - - /** - * Touch a file - * @param fs filesystem - * @param path path - * @throws IOException IO problems - */ - public static void touch(FileSystem fs, - Path path) throws IOException { - createFile(fs, path, true, null); - } - - /** - * Delete a file/dir and assert that delete() returned true - * and that the path no longer exists. This variant rejects - * all operations on root directories - * @param fs filesystem - * @param file path to delete - * @param recursive flag to enable recursive delete - * @throws IOException IO problems - */ - public static void assertDeleted(FileSystem fs, - Path file, - boolean recursive) throws IOException { - assertDeleted(fs, file, recursive, false); - } - - /** - * Delete a file/dir and assert that delete() returned true - * and that the path no longer exists. This variant rejects - * all operations on root directories - * @param fs filesystem - * @param file path to delete - * @param recursive flag to enable recursive delete - * @param allowRootOperations can the root dir be deleted? - * @throws IOException IO problems - */ - public static void assertDeleted(FileSystem fs, - Path file, - boolean recursive, - boolean allowRootOperations) throws IOException { - rejectRootOperation(file, allowRootOperations); - assertPathExists(fs, "about to be deleted file", file); - boolean deleted = fs.delete(file, recursive); - String dir = ls(fs, file.getParent()); - assertTrue("Delete failed on " + file + ": " + dir, deleted); - assertPathDoesNotExist(fs, "Deleted file", file); - } - - /** - * Read in "length" bytes, convert to an ascii string - * @param fs filesystem - * @param path path to read - * @param length #of bytes to read. - * @return the bytes read and converted to a string - * @throws IOException IO problems - */ - public static String readBytesToString(FileSystem fs, - Path path, - int length) throws IOException { - FSDataInputStream in = fs.open(path); - try { - byte[] buf = new byte[length]; - in.readFully(0, buf); - return toChar(buf); - } finally { - in.close(); - } - } - - /** - * Take an array of filestats and convert to a string (prefixed w/ a [01] counter - * @param stats array of stats - * @param separator separator after every entry - * @return a stringified set - */ - public static String fileStatsToString(FileStatus[] stats, String separator) { - StringBuilder buf = new StringBuilder(stats.length * 128); - for (int i = 0; i < stats.length; i++) { - buf.append(String.format("[%02d] %s", i, stats[i])).append(separator); - } - return buf.toString(); - } - - /** - * List a directory - * @param fileSystem FS - * @param path path - * @return a directory listing or failure message - * @throws IOException - */ - public static String ls(FileSystem fileSystem, Path path) throws IOException { - if (path == null) { - //surfaces when someone calls getParent() on something at the top of the path - return "/"; - } - FileStatus[] stats; - String pathtext = "ls " + path; - try { - stats = fileSystem.listStatus(path); - } catch (FileNotFoundException e) { - return pathtext + " -file not found"; - } catch (IOException e) { - return pathtext + " -failed: " + e; - } - return dumpStats(pathtext, stats); - } - - public static String dumpStats(String pathname, FileStatus[] stats) { - return pathname + fileStatsToString(stats, "\n"); - } - - /** - * Assert that a file exists and whose {@link FileStatus} entry - * declares that this is a file and not a symlink or directory. - * @param fileSystem filesystem to resolve path against - * @param filename name of the file - * @throws IOException IO problems during file operations - */ - public static void assertIsFile(FileSystem fileSystem, Path filename) throws - IOException { - assertPathExists(fileSystem, "Expected file", filename); - FileStatus status = fileSystem.getFileStatus(filename); - assertIsFile(filename, status); - } - - /** - * Assert that a file exists and whose {@link FileStatus} entry - * declares that this is a file and not a symlink or directory. - * @param filename name of the file - * @param status file status - */ - public static void assertIsFile(Path filename, FileStatus status) { - String fileInfo = filename + " " + status; - assertFalse("File claims to be a directory " + fileInfo, - status.isDirectory()); - assertFalse("File claims to be a symlink " + fileInfo, - status.isSymlink()); - } - - /** - * Create a dataset for use in the tests; all data is in the range - * base to (base+modulo-1) inclusive - * @param len length of data - * @param base base of the data - * @param modulo the modulo - * @return the newly generated dataset - */ - public static byte[] dataset(int len, int base, int modulo) { - byte[] dataset = new byte[len]; - for (int i = 0; i < len; i++) { - dataset[i] = (byte) (base + (i % modulo)); - } - return dataset; - } - - /** - * Assert that a path exists -but make no assertions as to the - * type of that entry - * - * @param fileSystem filesystem to examine - * @param message message to include in the assertion failure message - * @param path path in the filesystem - * @throws FileNotFoundException raised if the path is missing - * @throws IOException IO problems - */ - public static void assertPathExists(FileSystem fileSystem, String message, - Path path) throws IOException { - if (!fileSystem.exists(path)) { - //failure, report it - String listing = ls(fileSystem, path.getParent()); - throw new FileNotFoundException(message + ": not found " + path - + " in \"" + path.getParent() + "\" :\n" + listing); - } - } - - /** - * Assert that a path does not exist - * - * @param fileSystem filesystem to examine - * @param message message to include in the assertion failure message - * @param path path in the filesystem - * @throws IOException IO problems - */ - public static void assertPathDoesNotExist(FileSystem fileSystem, - String message, - Path path) throws IOException { - try { - FileStatus status = fileSystem.getFileStatus(path); - fail(message + ": unexpectedly found " + path + " as " + status); - } catch (FileNotFoundException expected) { - //this is expected - - } - } - - /** - * Assert that a FileSystem.listStatus on a dir finds the subdir/child entry - * @param fs filesystem - * @param dir directory to scan - * @param subdir full path to look for - * @throws IOException IO probles - */ - public static void assertListStatusFinds(FileSystem fs, - Path dir, - Path subdir) throws IOException { - FileStatus[] stats = fs.listStatus(dir); - boolean found = false; - StringBuilder builder = new StringBuilder(); - for (FileStatus stat : stats) { - builder.append(stat.toString()).append('\n'); - if (stat.getPath().equals(subdir)) { - found = true; - } - } - assertTrue("Path " + subdir - + " not found in directory " + dir + ":" + builder, - found); - } - - /** - * Test for the host being an OSX machine - * @return true if the JVM thinks that is running on OSX - */ - public static boolean isOSX() { - return System.getProperty("os.name").contains("OS X"); - } - - /** - * compare content of file operations using a double byte array - * @param concat concatenated files - * @param bytes bytes - */ - public static void validateFileContent(byte[] concat, byte[][] bytes) { - int idx = 0; - boolean mismatch = false; - - for (byte[] bb : bytes) { - for (byte b : bb) { - if (b != concat[idx++]) { - mismatch = true; - break; - } - } - if (mismatch) - break; - } - assertFalse("File content of file is not as expected at offset " + idx, - mismatch); - } - - /** - * Receives test data from the given input file and checks the size of the - * data as well as the pattern inside the received data. - * - * @param fs FileSystem - * @param path Input file to be checked - * @param expectedSize the expected size of the data to be read from the - * input file in bytes - * @param bufferLen Pattern length - * @param modulus Pattern modulus - * @throws IOException - * thrown if an error occurs while reading the data - */ - public static void verifyReceivedData(FileSystem fs, Path path, - final long expectedSize, - final int bufferLen, - final int modulus) throws IOException { - final byte[] testBuffer = new byte[bufferLen]; - - long totalBytesRead = 0; - int nextExpectedNumber = 0; - final InputStream inputStream = fs.open(path); - try { - while (true) { - final int bytesRead = inputStream.read(testBuffer); - if (bytesRead < 0) { - break; - } - - totalBytesRead += bytesRead; - - for (int i = 0; i < bytesRead; ++i) { - if (testBuffer[i] != nextExpectedNumber) { - throw new IOException("Read number " + testBuffer[i] - + " but expected " + nextExpectedNumber); - } - - ++nextExpectedNumber; - - if (nextExpectedNumber == modulus) { - nextExpectedNumber = 0; - } - } - } - - if (totalBytesRead != expectedSize) { - throw new IOException("Expected to read " + expectedSize + - " bytes but only received " + totalBytesRead); - } - } finally { - inputStream.close(); - } - } - - /** - * Generates test data of the given size according to some specific pattern - * and writes it to the provided output file. - * - * @param fs FileSystem - * @param path Test file to be generated - * @param size The size of the test data to be generated in bytes - * @param bufferLen Pattern length - * @param modulus Pattern modulus - * @throws IOException - * thrown if an error occurs while writing the data - */ - public static long generateTestFile(FileSystem fs, Path path, - final long size, - final int bufferLen, - final int modulus) throws IOException { - final byte[] testBuffer = new byte[bufferLen]; - for (int i = 0; i < testBuffer.length; ++i) { - testBuffer[i] = (byte) (i % modulus); - } - - final OutputStream outputStream = fs.create(path, false); - long bytesWritten = 0; - try { - while (bytesWritten < size) { - final long diff = size - bytesWritten; - if (diff < testBuffer.length) { - outputStream.write(testBuffer, 0, (int) diff); - bytesWritten += diff; - } else { - outputStream.write(testBuffer); - bytesWritten += testBuffer.length; - } - } - - return bytesWritten; - } finally { - outputStream.close(); - } - } - - /** - * Creates and reads a file with the given size. The test file is generated - * according to a specific pattern so it can be easily verified even if it's - * a multi-GB one. - * During the read phase the incoming data stream is also checked against - * this pattern. - * - * @param fs FileSystem - * @param parent Test file parent dir path - * @throws IOException - * thrown if an I/O error occurs while writing or reading the test file - */ - public static void createAndVerifyFile(FileSystem fs, Path parent, final long fileSize) - throws IOException { - int testBufferSize = fs.getConf() - .getInt(IO_CHUNK_BUFFER_SIZE, DEFAULT_IO_CHUNK_BUFFER_SIZE); - int modulus = fs.getConf() - .getInt(IO_CHUNK_MODULUS_SIZE, DEFAULT_IO_CHUNK_MODULUS_SIZE); - - final String objectName = UUID.randomUUID().toString(); - final Path objectPath = new Path(parent, objectName); - - // Write test file in a specific pattern - assertEquals(fileSize, - generateTestFile(fs, objectPath, fileSize, testBufferSize, modulus)); - assertPathExists(fs, "not created successful", objectPath); - - // Now read the same file back and verify its content - try { - verifyReceivedData(fs, objectPath, fileSize, testBufferSize, modulus); - } finally { - // Delete test file - fs.delete(objectPath, false); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/KeysForTests.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/KeysForTests.java deleted file mode 100644 index 06673d2..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/KeysForTests.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.slider.utils; - -import org.apache.hadoop.yarn.service.conf.SliderKeys; -import org.apache.slider.common.SliderXMLConfKeysForTesting; - -/** - * Keys shared across tests. - */ -public interface KeysForTests extends SliderKeys, SliderXMLConfKeysForTesting { - /** - * Username for all clusters, ZK, etc. - */ - String USERNAME = "bigdataborat"; - - int WAIT_TIME = 120; - String WAIT_TIME_ARG = Integer.toString(WAIT_TIME); - - String SLIDER_TEST_XML = "slider-test.xml"; - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/MicroZKCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/MicroZKCluster.java deleted file mode 100644 index be452f1..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/MicroZKCluster.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * 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.slider.utils; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.registry.client.api.RegistryOperations; -import org.apache.hadoop.registry.client.impl.zk.RegistryOperationsService; -import org.apache.hadoop.registry.server.services.MicroZookeeperService; -import org.apache.slider.common.tools.SliderUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.Closeable; -import java.io.IOException; - -/** - * Test ZK cluster. - */ -public class MicroZKCluster implements Closeable { - private static final Logger LOG = - LoggerFactory.getLogger(MicroZKCluster.class); - - public static final String HOSTS = "127.0.0.1"; - private MicroZookeeperService zkService; - private String zkBindingString; - private final Configuration conf; - private RegistryOperations registryOperations; - - MicroZKCluster() { - this(SliderUtils.createConfiguration()); - } - - MicroZKCluster(Configuration conf) { - this.conf = conf; - } - - String getZkBindingString() { - return zkBindingString; - } - - void createCluster(String name) { - zkService = new MicroZookeeperService(name); - - zkService.init(conf); - zkService.start(); - zkBindingString = zkService.getConnectionString(); - LOG.info("Created {}", this); - registryOperations = new RegistryOperationsService( - "registry", - zkService); - registryOperations.init(conf); - registryOperations.start(); - } - - @Override - public void close() throws IOException { - if (registryOperations != null) { - registryOperations.stop(); - } - if (zkService != null) { - zkService.stop(); - } - } - - @Override - public String toString() { - return "Micro ZK cluster as " + zkBindingString; - } - - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/Outcome.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/Outcome.java deleted file mode 100644 index 52875d3..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/Outcome.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.slider.utils; - -/** - * Outcome for probes. - */ -public final class Outcome { - - private final String name; - - private Outcome(String name) { - this.name = name; - } - - public static final Outcome SUCCESS = new Outcome( - "Success"); - public static final Outcome RETRY = new Outcome("Retry"); - public static final Outcome FAIL = new Outcome("Fail"); - - /** - * Build from a bool, where false is mapped to retry. - * @param b boolean - * @return an outcome - */ - static Outcome fromBool(boolean b) { - return b ? SUCCESS : RETRY; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/SliderTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/SliderTestBase.java deleted file mode 100644 index f7da585..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/SliderTestBase.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.slider.utils; - -import org.apache.hadoop.fs.FileUtil; -import org.apache.slider.common.SliderXMLConfKeysForTesting; -import org.apache.slider.server.appmaster.management.MetricsAndMonitoring; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.rules.TestName; - -import java.io.File; - - -/** - * Base class for unit tests as well as ones starting mini clusters - * -the foundational code and methods. - * - */ -public abstract class SliderTestBase extends SliderTestUtils { - - /** - * Singleton metric registry. - */ - public static final MetricsAndMonitoring METRICS = new MetricsAndMonitoring(); - public static final int WEB_STARTUP_TIME = 30000; - - @Rule - public TestName methodName = new TestName(); - - @BeforeClass - public static void nameThread() { - Thread.currentThread().setName("JUnit"); - } - - @Before - public void setup() throws Exception { - setSliderClientClassName(DEFAULT_SLIDER_CLIENT); - FileUtil.fullyDelete(new File(SliderXMLConfKeysForTesting - .TEST_SECURITY_DIR)); - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/SliderTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/SliderTestUtils.java deleted file mode 100644 index 50d56b0..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/SliderTestUtils.java +++ /dev/null @@ -1,1065 +0,0 @@ -/* - * 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.slider.utils; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.ObjectReader; -import com.fasterxml.jackson.databind.ObjectWriter; -import com.google.gson.Gson; -import com.google.gson.GsonBuilder; -import com.google.gson.JsonParser; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.registry.client.types.ServiceRecord; -import org.apache.hadoop.service.ServiceStateException; -import org.apache.hadoop.util.Shell; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.slider.api.resource.Application; -import org.apache.slider.api.resource.Container; -import org.apache.slider.client.SliderClient; -import org.apache.hadoop.yarn.service.client.params.Arguments; -import org.apache.slider.common.tools.Duration; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.main.LauncherExitCodes; -import org.apache.slider.core.main.ServiceLaunchException; -import org.apache.slider.core.main.ServiceLauncher; -import org.apache.slider.core.persist.JsonSerDeser; -import org.apache.slider.core.registry.docstore.PublishedConfigSet; -import org.apache.slider.core.registry.docstore.PublishedConfiguration; -import org.apache.slider.server.services.workflow.ForkedProcessService; -import org.codehaus.jackson.map.PropertyNamingStrategy; -import org.junit.Assert; -import org.junit.Assume; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Map.Entry; -import java.util.concurrent.TimeoutException; - -import static org.apache.hadoop.yarn.service.client.params.Arguments.ARG_OPTION; - -/** - * Static utils for tests in this package and in other test projects. - * - * It is designed to work with mini clusters as well as remote ones - * - * This class is not final and may be extended for test cases. - * - * Some of these methods are derived from the SwiftUtils and SwiftTestUtils - * classes -replicated here so that they are available in Hadoop-2.0 code - */ -public class SliderTestUtils extends Assert { - private static final Logger LOG = - LoggerFactory.getLogger(SliderTestUtils.class); - public static final String DEFAULT_SLIDER_CLIENT = SliderClient.class - .getName(); - private static String sliderClientClassName = DEFAULT_SLIDER_CLIENT; - - public static final Map EMPTY_MAP = Collections.emptyMap(); - public static final Map EMPTY_INT_MAP = Collections - .emptyMap(); - public static final List EMPTY_LIST = Collections.emptyList(); - - public static final ObjectReader OBJECT_READER; - public static final ObjectWriter OBJECT_WRITER; - - public static final JsonSerDeser JSON_SER_DESER = - new JsonSerDeser<>(Application.class, - PropertyNamingStrategy.CAMEL_CASE_TO_LOWER_CASE_WITH_UNDERSCORES); - - static { - ObjectMapper mapper = new ObjectMapper(); - OBJECT_READER = mapper.readerFor(Object.class); - OBJECT_WRITER = mapper.writer(); - } - - /** - * Action that returns an object. - */ - public interface Action { - Object invoke() throws Exception; - } - - /** - * Probe that returns an Outcome. - */ - public interface Probe { - Outcome invoke(Map args) throws Exception; - } - - public static void setSliderClientClassName(String sliderClientClassName) { - sliderClientClassName = sliderClientClassName; - } - - public static void describe(String s) { - LOG.info(""); - LOG.info("==============================="); - LOG.info(s); - LOG.info("==============================="); - LOG.info(""); - } - - /** - * Convert a JSON string to something readable. - * @param json - * @return a string for printing - */ - public static String prettyPrintJson(String json) { - Gson gson = new GsonBuilder().setPrettyPrinting().create(); - return gson.toJson(new JsonParser().parse(json)); - } - - /** - * Convert an object to something readable. - * @param src - * @return a string for printing - */ - public static String prettyPrintAsJson(Object src) - throws JsonProcessingException, UnsupportedEncodingException { - return new String(OBJECT_WRITER.writeValueAsBytes(src), "UTF8"); - } - - /** - * Skip the test with a message. - * @param message message logged and thrown - */ - public static void skip(String message) { - LOG.warn("Skipping test: {}", message); - Assume.assumeTrue(message, false); - } - - /** - * Skip the test with a message if condition holds. - * @param condition predicate - * @param message message logged and thrown - */ - public static void assume(boolean condition, String message) { - if (!condition) { - skip(message); - } - } - - /** - * Skip a test if not running on Windows. - */ - public static void assumeWindows() { - assume(Shell.WINDOWS, "not windows"); - } - - /** - * Skip a test if running on Windows. - */ - public static void assumeNotWindows() { - assume(!Shell.WINDOWS, "windows"); - } - - /** - * Skip a test on windows. - */ - public static void skipOnWindows() { - assumeNotWindows(); - } - - /** - * Equality size for a list. - * @param left - * @param right - */ - public static void assertListEquals(List left, List right) { - String lval = collectionToString(left); - String rval = collectionToString(right); - String text = "comparing " + lval + " to " + rval; - assertEquals(text, left.size(), right.size()); - for (int i = 0; i < left.size(); i++) { - assertEquals(text, left.get(i), right.get(i)); - } - } - - /** - * Assert a list has a given length. - * @param list list - * @param size size to have - */ - public static void assertListLength(List list, int size) { - String lval = collectionToString(list); - assertEquals(lval, size, list.size()); - } - - /** - * Stringify a collection with [ ] at either end. - * @param collection collection - * @return string value - */ - public static String collectionToString(List collection) { - return "[" + SliderUtils.join(collection, ", ", false) + "]"; - } - - /** - * Assume that a string option is set and not equal to "". - * @param conf configuration file - * @param key key to look for - */ - public static void assumeStringOptionSet(Configuration conf, String key) { - if (SliderUtils.isUnset(conf.getTrimmed(key))) { - skip("Configuration key " + key + " not set"); - } - } - - /** - * assert that a string option is set and not equal to "". - * @param conf configuration file - * @param key key to look for - */ - public static void assertStringOptionSet(Configuration conf, String key) { - getRequiredConfOption(conf, key); - } - - /** - * Assume that a boolean option is set and true. - * Unset or false triggers a test skip - * @param conf configuration file - * @param key key to look for - */ - public static void assumeBoolOptionTrue(Configuration conf, String key) { - assumeBoolOption(conf, key, false); - } - - /** - * Assume that a boolean option is true. - * False triggers a test skip - * @param conf configuration file - * @param key key to look for - * @param defval default value if the property is not defined - */ - public static void assumeBoolOption( - Configuration conf, String key, boolean defval) { - assume(conf.getBoolean(key, defval), - "Configuration key " + key + " is false"); - } - - /** - * Get a required config option (trimmed, incidentally). - * Test will fail if not set - * @param conf configuration - * @param key key - * @return the string - */ - public static String getRequiredConfOption(Configuration conf, String key) { - String val = conf.getTrimmed(key); - if (SliderUtils.isUnset(val)) { - fail("Missing configuration option " + key); - } - return val; - } - - /** - * Fails a test because required behavior has not been implemented. - */ - public static void failNotImplemented() { - fail("Not implemented"); - } - - /** - * Assert that any needed libraries being present. On Unix none are needed; - * on windows they must be present - */ - public static void assertNativeLibrariesPresent() { - String errorText = SliderUtils.checkForRequiredNativeLibraries(); - if (SliderUtils.isSet(errorText)) { - fail(errorText); - } - } - - protected static String[] toArray(List args) { - String[] converted = new String[args.size()]; - for (int i = 0; i < args.size(); i++) { - Object elt = args.get(i); - assertNotNull(args.get(i)); - converted[i] = elt.toString(); - } - return converted; - } - - public static void waitWhileClusterLive(SliderClient client, int timeout) - throws IOException, YarnException { - Duration duration = new Duration(timeout); - duration.start(); - while (client.actionExists(client.getDeployedClusterName(), true) == - LauncherExitCodes.EXIT_SUCCESS && !duration.getLimitExceeded()) { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - } - } - if (duration.getLimitExceeded()) { - fail("Cluster " + client.getDeployedClusterName() + " still live after " + - timeout + " ms"); - } - } - - public static void waitUntilClusterLive(SliderClient client, int timeout) - throws IOException, YarnException { - Duration duration = new Duration(timeout); - duration.start(); - while (LauncherExitCodes.EXIT_SUCCESS != client.actionExists( - client.getDeployedClusterName(), true) && - !duration.getLimitExceeded()) { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - } - } - if (duration.getLimitExceeded()) { - fail("Cluster " + client.getDeployedClusterName() + " not live after " + - timeout + " ms"); - } - } - - public static void dumpClusterDescription( - String text, - Application status) throws IOException { - describe(text); - LOG.info(JSON_SER_DESER.toJson(status)); - } - - /** - * Assert that a service operation succeeded. - * @param service service - */ - public static void assertSucceeded(ServiceLauncher service) { - assertEquals(0, service.getServiceExitCode()); - } - - public static void assertContainersLive(Application application, - String component, int expected) { - LOG.info("Asserting component {} expected count {}", component, expected); - int actual = extractLiveContainerCount(application, component); - if (expected != actual) { - LOG.warn("{} actual={}, expected {} in \n{}\n", component, actual, - expected, application); - } - assertEquals(expected, actual); - } - - /** - * Robust extraction of live container count. - * @param application status - * @param component component to resolve - * @return the number of containers live. - */ - public static int extractLiveContainerCount( - Application application, - String component) { - int actual = 0; - if (application.getContainers() != null) { - for (Container container : application.getContainers()) { - if (container.getComponentName().equals(component)) { - actual++; - } - } - } - return actual; - } - - /** - * Exec a set of commands, wait a few seconds for it to finish. - * @param status code - * @param commands - * @return the process - */ - public static ForkedProcessService exec(int status, List commands) - throws IOException, TimeoutException { - ForkedProcessService process = exec(commands); - - Integer exitCode = process.getExitCode(); - assertNotNull(exitCode); - assertEquals(status, exitCode.intValue()); - return process; - } - - /** - * Exec a set of commands, wait a few seconds for it to finish. - * @param commands - * @return - */ - public static ForkedProcessService exec(List commands) - throws IOException, TimeoutException { - ForkedProcessService process; - process = new ForkedProcessService( - commands.get(0), - EMPTY_MAP, - commands); - process.init(new Configuration()); - process.start(); - int timeoutMillis = 5000; - if (!process.waitForServiceToStop(timeoutMillis)) { - throw new TimeoutException( - "Process did not stop in " + timeoutMillis + "mS"); - } - return process; - } - - /** - * Determine whether an application exists. Run the commands and if the - * operation fails with a FileNotFoundException, then - * this method returns false. - *

- * Run something harmless like a -version command, something - * which must return 0 - * - * @param commands - * @return true if the command sequence succeeded - * false if they failed with no file - * @throws Exception on any other failure cause - */ - public static boolean doesAppExist(List commands) - throws IOException, TimeoutException { - try { - exec(0, commands); - return true; - } catch (ServiceStateException e) { - if (!(e.getCause() instanceof FileNotFoundException)) { - throw e; - } - return false; - } - } - - /** - * Locate an executable on the path. - * @param exe executable name. If it is an absolute path which - * exists then it will returned direct - * @return the path to an exe or null for no match - */ - public static File locateExecutable(String exe) { - File exeNameAsPath = new File(exe).getAbsoluteFile(); - if (exeNameAsPath.exists()) { - return exeNameAsPath; - } - - File exepath = null; - String path = extractPath(); - String[] dirs = path.split(System.getProperty("path.separator")); - for (String dirname : dirs) { - File dir = new File(dirname); - - File possible = new File(dir, exe); - if (possible.exists()) { - exepath = possible; - } - } - return exepath; - } - - /** - * Lookup the PATH env var. - * @return the path or null - */ - public static String extractPath() { - return extractEnvVar("PATH"); - } - - /** - * Find an environment variable. Uses case independent checking for - * the benefit of windows. - * Will fail if the var is not found. - * @param var path variable in upper case - * @return the env var - */ - public static String extractEnvVar(String var) { - String realkey = ""; - - for (String it : System.getenv().keySet()) { - if (it.toUpperCase(Locale.ENGLISH).equals(var)) { - realkey = it; - } - } - - if (SliderUtils.isUnset(realkey)) { - fail("No environment variable " + var + " found"); - } - String val = System.getenv(realkey); - - LOG.info("{} = {}", realkey, val); - return val; - } - - /** - * Create a temp JSON file. After coming up with the name, the file - * is deleted - * @return the filename - */ - public static File createTempJsonFile() throws IOException { - return tmpFile(".json"); - } - - /** - * Create a temp file with the specific name. It's deleted after creation, - * to avoid "file exists exceptions" - * @param suffix suffix, e.g. ".txt" - * @return a path to a file which may be created - */ - public static File tmpFile(String suffix) throws IOException { - File reportFile = File.createTempFile( - "temp", - suffix, - new File("target")); - reportFile.delete(); - return reportFile; - } - - /** - * Execute a closure, assert it fails with a given exit code and text. - * @param exitCode exit code - * @param text text (can be "") - * @param action action - * @return - */ - public void assertFailsWithException(int exitCode, - String text, - Action action) throws Exception { - try { - action.invoke(); - fail("Operation was expected to fail —but it succeeded"); - } catch (ServiceLaunchException e) { - assertExceptionDetails(e, exitCode, text); - } - } - - /** - * Execute a closure, assert it fails with a given exit code and text. - * @param text text (can be "") - * @param action action - * @return - */ - public void assertFailsWithExceptionClass(Class clazz, - String text, - Action action) throws Exception { - try { - action.invoke(); - fail("Operation was expected to fail —but it succeeded"); - } catch (Exception e) { - assertExceptionDetails(e, clazz, text); - } - } - - public static void assertExceptionDetails( - ServiceLaunchException ex, - int exitCode) { - assertExceptionDetails(ex, exitCode, null); - } - - /** - * Make an assertion about the exit code of an exception. - * @param ex exception - * @param exitCode exit code - * @param text error text to look for in the exception - */ - public static void assertExceptionDetails( - ServiceLaunchException ex, - int exitCode, - String text) { - if (exitCode != ex.getExitCode()) { - String message = String.format("Wrong exit code, expected %d but" + - " got %d in %s", exitCode, ex.getExitCode(), ex); - LOG.warn(message, ex); - throw new AssertionError(message, ex); - } - if (SliderUtils.isSet(text)) { - if (!(ex.toString().contains(text))) { - String message = String.format("String match for \"%s\"failed in %s", - text, ex); - LOG.warn(message, ex); - throw new AssertionError(message, ex); - } - } - } - - /** - * Make an assertion about the class of an exception. - * @param ex exception - * @param clazz exit code - * @param text error text to look for in the exception - */ - static void assertExceptionDetails( - Exception ex, - Class clazz, - String text) throws Exception { - if (ex.getClass() != clazz) { - throw ex; - } - if (SliderUtils.isSet(text) && !(ex.toString().contains(text))) { - throw ex; - } - } - - /** - * Launch the slider client with the specific args; no validation - * of return code takes place. - * @param conf configuration - * @param args arg list - * @return the launcher - */ - protected static ServiceLauncher execSliderCommand( - Configuration conf, - List args) throws Throwable { - ServiceLauncher serviceLauncher = - new ServiceLauncher<>(sliderClientClassName); - - LOG.debug("slider {}", SliderUtils.join(args, " ", false)); - serviceLauncher.launchService(conf, - toArray(args), - false); - return serviceLauncher; - } - - /** - * Launch a slider command to a given exit code. - * Most failures will trigger exceptions; this is for the exit code of the - * runService() call. - * @param exitCode desired exit code - * @param conf configuration - * @param args arg list - * @return the launcher - */ - protected static ServiceLauncher execSliderCommand( - int exitCode, - Configuration conf, - List args) throws Throwable { - ServiceLauncher serviceLauncher = execSliderCommand(conf, - args); - assertEquals(exitCode, serviceLauncher.getServiceExitCode()); - return serviceLauncher; - } - - public static ServiceLauncher launch(Class serviceClass, - Configuration conf, - List args) throws - Throwable { - ServiceLauncher serviceLauncher = - new ServiceLauncher(serviceClass.getName()); - - String joinedArgs = SliderUtils.join(args, " ", false); - LOG.debug("slider {}", joinedArgs); - - serviceLauncher.launchService(conf, - toArray(args), - false); - return serviceLauncher; - } - - public static Throwable launchExpectingException(Class serviceClass, - Configuration conf, - String expectedText, - List args) - throws Throwable { - try { - ServiceLauncher launch = launch(serviceClass, conf, args); - throw new AssertionError("Expected an exception with text containing " + - expectedText + " -but the service completed with exit code " + - launch.getServiceExitCode()); - } catch (AssertionError error) { - throw error; - } catch (Throwable thrown) { - if (SliderUtils.isSet(expectedText) && !thrown.toString().contains( - expectedText)) { - //not the right exception -rethrow - LOG.warn("Caught Exception did not contain expected text" + - "\"" + expectedText + "\""); - throw thrown; - } - return thrown; - } - } - - - public static ServiceLauncher launchClientAgainstRM( - String address, - List args, - Configuration conf) throws Throwable { - assertNotNull(address); - LOG.info("Connecting to rm at {}", address); - if (!args.contains(Arguments.ARG_MANAGER)) { - args.add(Arguments.ARG_MANAGER); - args.add(address); - } - ServiceLauncher launcher = execSliderCommand(conf, args); - return launcher; - } - - /** - * Add a configuration parameter as a cluster configuration option. - * @param extraArgs extra arguments - * @param conf config - * @param option option - */ - public static void addClusterConfigOption( - List extraArgs, - YarnConfiguration conf, - String option) { - - conf.getTrimmed(option); - extraArgs.add(ARG_OPTION); - extraArgs.add(option); - extraArgs.add(getRequiredConfOption(conf, option)); - } - - /** - * Assert that a path refers to a directory. - * @param fs filesystem - * @param path path of the directory - * @throws IOException on File IO problems - */ - public static void assertIsDirectory(FileSystem fs, - Path path) throws IOException { - FileStatus fileStatus = fs.getFileStatus(path); - assertIsDirectory(fileStatus); - } - - /** - * Assert that a path refers to a directory. - * @param fileStatus stats to check - */ - public static void assertIsDirectory(FileStatus fileStatus) { - assertTrue("Should be a dir -but isn't: " + fileStatus, - fileStatus.isDirectory()); - } - - /** - * Assert that a path exists -but make no assertions as to the - * type of that entry. - * - * @param fileSystem filesystem to examine - * @param message message to include in the assertion failure message - * @param path path in the filesystem - * @throws IOException IO problems - */ - public static void assertPathExists( - FileSystem fileSystem, - String message, - Path path) throws IOException { - if (!fileSystem.exists(path)) { - //failure, report it - fail( - message + ": not found \"" + path + "\" in " + path.getParent() + - "-" + - ls(fileSystem, path.getParent())); - } - } - - /** - * Assert that a path does not exist. - * - * @param fileSystem filesystem to examine - * @param message message to include in the assertion failure message - * @param path path in the filesystem - * @throws IOException IO problems - */ - public static void assertPathDoesNotExist( - FileSystem fileSystem, - String message, - Path path) throws IOException { - try { - FileStatus status = fileSystem.getFileStatus(path); - // a status back implies there is a file here - fail(message + ": unexpectedly found " + path + " as " + status); - } catch (FileNotFoundException expected) { - //this is expected - - } - } - - /** - * Assert that a FileSystem.listStatus on a dir finds the subdir/child entry. - * @param fs filesystem - * @param dir directory to scan - * @param subdir full path to look for - * @throws IOException IO probles - */ - public static void assertListStatusFinds(FileSystem fs, - Path dir, - Path subdir) throws IOException { - FileStatus[] stats = fs.listStatus(dir); - boolean found = false; - StringBuilder builder = new StringBuilder(); - for (FileStatus stat : stats) { - builder.append(stat.toString()).append('\n'); - if (stat.getPath().equals(subdir)) { - found = true; - } - } - assertTrue("Path " + subdir - + " not found in directory " + dir + ":" + builder, - found); - } - - /** - * List a a path to string. - * @param fileSystem filesystem - * @param path directory - * @return a listing of the filestatuses of elements in the directory, one - * to a line, precedeed by the full path of the directory - * @throws IOException connectivity problems - */ - public static String ls(FileSystem fileSystem, Path path) - throws IOException { - if (path == null) { - //surfaces when someone calls getParent() on something at the top of - // the path - return "/"; - } - FileStatus[] stats; - String pathtext = "ls " + path; - try { - stats = fileSystem.listStatus(path); - } catch (FileNotFoundException e) { - return pathtext + " -file not found"; - } catch (IOException e) { - return pathtext + " -failed: " + e; - } - return pathtext + fileStatsToString(stats, "\n"); - } - - /** - * Take an array of filestats and convert to a string (prefixed w/ a [01] - * counter). - * @param stats array of stats - * @param separator separator after every entry - * @return a stringified set - */ - public static String fileStatsToString(FileStatus[] stats, String separator) { - StringBuilder buf = new StringBuilder(stats.length * 128); - for (int i = 0; i < stats.length; i++) { - buf.append(String.format("[%02d] %s", i, stats[i])).append(separator); - } - return buf.toString(); - } - - public static void waitWhileClusterLive(SliderClient sliderClient) - throws IOException, YarnException { - waitWhileClusterLive(sliderClient, 30000); - } - - public static void dumpRegistryInstances( - Map instances) { - describe("service registry slider instances"); - for (Entry it : instances.entrySet()) { - LOG.info(" {} : {}", it.getKey(), it.getValue()); - } - describe("end list service registry slider instances"); - } - - - public static void dumpRegistryInstanceIDs(List instanceIds) { - describe("service registry instance IDs"); - dumpCollection(instanceIds); - } - - public static void dumpRegistryServiceTypes(Collection entries) { - describe("service registry types"); - dumpCollection(entries); - } - - public static void dumpCollection(Collection entries) { - LOG.info("number of entries: {}", entries.size()); - for (V it : entries) { - LOG.info(it.toString()); - } - } - - public static void dumpArray(Object[] entries) { - LOG.info("number of entries: {}", entries.length); - for (Object it : entries) { - LOG.info(it.toString()); - } - } - - public static void dumpMap(Map map) { - for (Entry it : map.entrySet()) { - LOG.info("\"{}\": \"{}\"", it.getKey().toString(), it.getValue() - .toString()); - } - } - - /** - * Get a time option in seconds if set, otherwise the default value (also - * in seconds). - * This operation picks up the time value as a system property if set -that - * value overrides anything in the test file - * @param conf - * @param key - * @param defValMillis - * @return - */ - public static int getTimeOptionMillis( - Configuration conf, - String key, - int defValMillis) { - int val = conf.getInt(key, 0); - val = Integer.getInteger(key, val); - int time = 1000 * val; - if (time == 0) { - time = defValMillis; - } - return time; - } - - public void dumpConfigurationSet(PublishedConfigSet confSet) { - for (String key : confSet.keys()) { - PublishedConfiguration config = confSet.get(key); - LOG.info("{} -- {}", key, config.description); - } - } - - /** - * Convert a file to a URI suitable for use in an argument. - * @param file file - * @return a URI string valid on all platforms - */ - public String toURIArg(File file) { - return file.getAbsoluteFile().toURI().toString(); - } - - /** - * Assert a file exists; fails with a listing of the parent dir. - * @param text text for front of message - * @param file file to look for - * @throws FileNotFoundException - */ - public void assertFileExists(String text, File file) - throws FileNotFoundException { - if (!file.exists()) { - File parent = file.getParentFile(); - String[] files = parent.list(); - StringBuilder builder = new StringBuilder(); - builder.append(parent.getAbsolutePath()); - builder.append(":\n"); - for (String name : files) { - builder.append(" "); - builder.append(name); - builder.append("\n"); - } - throw new FileNotFoundException(text + ": " + file + " not found in " + - builder); - } - } - - /** - * Repeat a probe until it succeeds, if it does not execute a failure - * closure then raise an exception with the supplied message. - * @param probe probe - * @param timeout time in millis before giving up - * @param sleepDur sleep between failing attempts - * @param args map of arguments to the probe - * @param failIfUnsuccessful if the probe fails after all the attempts - * —should it raise an exception - * @param failureMessage message to include in exception raised - * @param failureHandler closure to invoke prior to the failure being raised - */ - protected void repeatUntilSuccess( - String action, - Probe probe, - int timeout, - int sleepDur, - Map args, - boolean failIfUnsuccessful, - String failureMessage, - Action failureHandler) throws Exception { - LOG.debug("Probe {} timelimit {}", action, timeout); - if (timeout < 1000) { - fail("Timeout " + timeout + " too low: milliseconds are expected, not " + - "seconds"); - } - int attemptCount = 1; - boolean succeeded = false; - boolean completed = false; - Duration duration = new Duration(timeout); - duration.start(); - while (!completed) { - Outcome outcome = probe.invoke(args); - if (outcome.equals(Outcome.SUCCESS)) { - // success - LOG.debug("Success after {} attempt(s)", attemptCount); - succeeded = true; - completed = true; - } else if (outcome.equals(Outcome.RETRY)) { - // failed but retry possible - attemptCount++; - completed = duration.getLimitExceeded(); - if (!completed) { - LOG.debug("Attempt {} failed", attemptCount); - try { - Thread.sleep(sleepDur); - } catch (InterruptedException e) { - } - } - } else if (outcome.equals(Outcome.FAIL)) { - // fast fail - LOG.debug("Fast fail of probe"); - completed = true; - } - } - if (!succeeded) { - if (duration.getLimitExceeded()) { - LOG.info("probe timed out after {} and {} attempts", timeout, - attemptCount); - } - if (failureHandler != null) { - failureHandler.invoke(); - } - if (failIfUnsuccessful) { - fail(failureMessage); - } - } - } - - /** - * Get a value from a map; raise an assertion if it is not there. - * @param map map to look up - * @param key key - * @return the string value - */ - public String requiredMapValue(Map map, String key) { - assertNotNull(map.get(key)); - return map.get(key).toString(); - } - - public static void assertStringContains(String expected, String text) { - assertNotNull("null text", text); - if (!text.contains(expected)) { - String message = String.format("did not find %s in \"%s\"", expected, - text); - LOG.error(message); - fail(message); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/TestAssertions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/TestAssertions.java deleted file mode 100644 index 2dfb204..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/TestAssertions.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.slider.utils; - -import org.apache.slider.api.resource.Application; -import org.junit.Test; - -import java.util.Collections; - -/** - * Test for some of the command test base operations. - */ -public class TestAssertions { - - public static final String CLUSTER_JSON = "json/cluster.json"; - - //@Test - public void testNoInstances() throws Throwable { - Application application = new Application(); - application.setContainers(null); - SliderTestUtils.assertContainersLive(application, "example", 0); - } - - //@Test - public void testEmptyInstances() throws Throwable { - Application application = new Application(); - application.setContainers(Collections.emptyList()); - SliderTestUtils.assertContainersLive(application, "example", 0); - } - -// TODO test metrics retrieval -// //@Test -// public void testLiveInstances() throws Throwable { -// InputStream stream = getClass().getClassLoader().getResourceAsStream( -// CLUSTER_JSON); -// assertNotNull("could not load " + CLUSTER_JSON, stream); -// ClusterDescription liveCD = ClusterDescription.fromStream(stream); -// assertNotNull(liveCD); -// SliderTestUtils.assertContainersLive(liveCD, "SLEEP_LONG", 4); -// assertEquals((Integer) 1, liveCD.statistics.get("SLEEP_LONG").get( -// StatusKeys.STATISTICS_CONTAINERS_ANTI_AFFINE_PENDING)); -// } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/TestUtility.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/TestUtility.java deleted file mode 100644 index 5493198..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/TestUtility.java +++ /dev/null @@ -1,181 +0,0 @@ -/* - * 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.slider.utils; - -import org.apache.commons.compress.archivers.zip.ZipArchiveEntry; -import org.apache.commons.compress.archivers.zip.ZipArchiveOutputStream; -import org.apache.commons.compress.utils.IOUtils; -import org.junit.Assert; -import org.junit.rules.TemporaryFolder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; - -/** - * Various utility methods - * Byte comparison methods are from - * org.apache.hadoop.fs.contract.ContractTestUtils - */ -public class TestUtility { - protected static final Logger log = - LoggerFactory.getLogger(TestUtility.class); - - public static void addDir(File dirObj, ZipArchiveOutputStream zipFile, String prefix) throws IOException { - for (File file : dirObj.listFiles()) { - if (file.isDirectory()) { - addDir(file, zipFile, prefix + file.getName() + File.separator); - } else { - log.info("Adding to zip - " + prefix + file.getName()); - zipFile.putArchiveEntry(new ZipArchiveEntry(prefix + file.getName())); - IOUtils.copy(new FileInputStream(file), zipFile); - zipFile.closeArchiveEntry(); - } - } - } - - public static void zipDir(String zipFile, String dir) throws IOException { - File dirObj = new File(dir); - ZipArchiveOutputStream out = new ZipArchiveOutputStream(new FileOutputStream(zipFile)); - log.info("Creating : {}", zipFile); - try { - addDir(dirObj, out, ""); - } finally { - out.close(); - } - } - - public static String createAppPackage( - TemporaryFolder folder, String subDir, String pkgName, String srcPath) throws IOException { - String zipFileName; - File pkgPath = folder.newFolder(subDir); - File zipFile = new File(pkgPath, pkgName).getAbsoluteFile(); - zipFileName = zipFile.getAbsolutePath(); - TestUtility.zipDir(zipFileName, srcPath); - log.info("Created temporary zip file at {}", zipFileName); - return zipFileName; - } - - - /** - * Assert that tthe array original[0..len] and received[] are equal. - * A failure triggers the logging of the bytes near where the first - * difference surfaces. - * @param original source data - * @param received actual - * @param len length of bytes to compare - */ - public static void compareByteArrays(byte[] original, - byte[] received, - int len) { - Assert.assertEquals("Number of bytes read != number written", - len, received.length); - int errors = 0; - int first_error_byte = -1; - for (int i = 0; i < len; i++) { - if (original[i] != received[i]) { - if (errors == 0) { - first_error_byte = i; - } - errors++; - } - } - - if (errors > 0) { - String message = String.format(" %d errors in file of length %d", - errors, len); - log.warn(message); - // the range either side of the first error to print - // this is a purely arbitrary number, to aid user debugging - final int overlap = 10; - for (int i = Math.max(0, first_error_byte - overlap); - i < Math.min(first_error_byte + overlap, len); - i++) { - byte actual = received[i]; - byte expected = original[i]; - String letter = toChar(actual); - String line = String.format("[%04d] %2x %s\n", i, actual, letter); - if (expected != actual) { - line = String.format("[%04d] %2x %s -expected %2x %s\n", - i, - actual, - letter, - expected, - toChar(expected)); - } - log.warn(line); - } - Assert.fail(message); - } - } - /** - * Convert a byte to a character for printing. If the - * byte value is < 32 -and hence unprintable- the byte is - * returned as a two digit hex value - * @param b byte - * @return the printable character string - */ - public static String toChar(byte b) { - if (b >= 0x20) { - return Character.toString((char) b); - } else { - return String.format("%02x", b); - } - } - - /** - * Convert a buffer to a string, character by character - * @param buffer input bytes - * @return a string conversion - */ - public static String toChar(byte[] buffer) { - StringBuilder builder = new StringBuilder(buffer.length); - for (byte b : buffer) { - builder.append(toChar(b)); - } - return builder.toString(); - } - - public static byte[] toAsciiByteArray(String s) { - char[] chars = s.toCharArray(); - int len = chars.length; - byte[] buffer = new byte[len]; - for (int i = 0; i < len; i++) { - buffer[i] = (byte) (chars[i] & 0xff); - } - return buffer; - } - - /** - * Create a dataset for use in the tests; all data is in the range - * base to (base+modulo-1) inclusive - * @param len length of data - * @param base base of the data - * @param modulo the modulo - * @return the newly generated dataset - */ - public static byte[] dataset(int len, int base, int modulo) { - byte[] dataset = new byte[len]; - for (int i = 0; i < len; i++) { - dataset[i] = (byte) (base + (i % modulo)); - } - return dataset; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/YarnMiniClusterTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/YarnMiniClusterTestBase.java deleted file mode 100644 index 6cda9c1..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/YarnMiniClusterTestBase.java +++ /dev/null @@ -1,873 +0,0 @@ -/* - * 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.slider.utils; - -import org.apache.commons.io.FileUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.service.ServiceOperations; -import org.apache.hadoop.util.Shell; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.server.MiniYARNCluster; -import org.apache.slider.client.SliderClient; -import org.apache.hadoop.yarn.service.conf.SliderExitCodes; -import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys; -import org.apache.slider.common.params.ActionFreezeArgs; -import org.apache.hadoop.yarn.service.client.params.Arguments; -import org.apache.hadoop.yarn.service.client.params.SliderActions; -import org.apache.slider.common.tools.Duration; -import org.apache.slider.common.tools.SliderFileSystem; -import org.apache.slider.common.tools.SliderUtils; -import org.apache.slider.core.exceptions.ErrorStrings; -import org.apache.slider.core.exceptions.SliderException; -import org.apache.slider.core.main.ServiceLauncher; -import org.apache.slider.server.appmaster.SliderAppMaster; -import org.junit.After; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.rules.Timeout; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.BufferedReader; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStreamReader; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Map.Entry; - -import static org.apache.slider.utils.KeysForTests.*; - -/** - * Base class for mini cluster tests -creates a field for the - * mini yarn cluster. - */ -public abstract class YarnMiniClusterTestBase extends SliderTestBase { - private static final Logger LOG = - LoggerFactory.getLogger(YarnMiniClusterTestBase.class); - - /** - * Mini YARN cluster only. - */ - public static final int CLUSTER_GO_LIVE_TIME = 3 * 60 * 1000; - public static final int CLUSTER_STOP_TIME = 1 * 60 * 1000; - public static final int SIGTERM = -15; - public static final int SIGKILL = -9; - public static final int SIGSTOP = -17; - public static final String NO_ARCHIVE_DEFINED = "Archive configuration " + - "option not set: "; - /** - * RAM for the YARN containers: {@value}. - */ - public static final String YRAM = "256"; - public static final String FIFO_SCHEDULER = "org.apache.hadoop.yarn.server" + - ".resourcemanager.scheduler.fifo.FifoScheduler"; - public static final YarnConfiguration SLIDER_CONFIG = - SliderUtils.createConfiguration(); - private static boolean killSupported; - - static { - SLIDER_CONFIG.setInt(SliderXmlConfKeys.KEY_AM_RESTART_LIMIT, 1); - SLIDER_CONFIG.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 100); - SLIDER_CONFIG.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false); - SLIDER_CONFIG.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false); - SLIDER_CONFIG - .setBoolean(SliderXmlConfKeys.KEY_SLIDER_AM_DEPENDENCY_CHECKS_DISABLED, - true); - SLIDER_CONFIG - .setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 1); - } - - - private int thawWaitTime = DEFAULT_THAW_WAIT_TIME_SECONDS * 1000; - private int freezeWaitTime = DEFAULT_TEST_FREEZE_WAIT_TIME_SECONDS * 1000; - private int sliderTestTimeout = DEFAULT_TEST_TIMEOUT_SECONDS * 1000; - private boolean teardownKillall = DEFAULT_TEARDOWN_KILLALL; - - /** - * This is set in a system property. - */ - @Rule - public Timeout testTimeout = new Timeout( - getTimeOptionMillis(getTestConfiguration(), - KEY_TEST_TIMEOUT, - DEFAULT_TEST_TIMEOUT_SECONDS * 1000) - ); - private MiniDFSCluster hdfsCluster; - private MiniYARNCluster miniCluster; - private boolean switchToImageDeploy = false; - private boolean imageIsRemote = false; - private URI remoteImageURI; - private List clustersToTeardown = new ArrayList<>(); - private int clusterCount = 1; - - /** - * Clent side test: validate system env before launch. - */ - @BeforeClass - public static void checkClientEnv() throws IOException, SliderException { - SliderUtils.validateSliderClientEnvironment(null); - } - - /** - * Work out if kill is supported. - */ - @BeforeClass - public static void checkKillSupport() { - killSupported = !Shell.WINDOWS; - } - - protected static boolean getKillSupported() { - return killSupported; - } - - protected MiniYARNCluster getMiniCluster() { - return miniCluster; - } - - /** - * Probe for the disks being healthy in a mini cluster. Only the first - * NM is checked. - * - * @param miniCluster - */ - public static void assertMiniClusterDisksHealthy( - MiniYARNCluster miniCluster) { - boolean healthy = miniCluster.getNodeManager( - 0).getNodeHealthChecker().getDiskHandler().areDisksHealthy(); - assertTrue("Disks on test cluster unhealthy —may be full", healthy); - } - - /** - * Inner work building the mini dfs cluster. - * - * @param name - * @param conf - * @return - */ - public static MiniDFSCluster buildMiniHDFSCluster( - String name, - YarnConfiguration conf) throws IOException { - assertNativeLibrariesPresent(); - - File baseDir = new File("./target/hdfs", name).getAbsoluteFile(); - //use file: to rm it recursively - FileUtil.fullyDelete(baseDir); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - - MiniDFSCluster cluster = builder.build(); - return cluster; - } - - public static String buildFsDefaultName(MiniDFSCluster miniDFSCluster) { - if (miniDFSCluster != null) { - return String.format("hdfs://localhost:%d/", - miniDFSCluster.getNameNodePort()); - } else { - return "file:///"; - } - } - - /** - * Assert that an operation failed because a cluster is in use. - * - * @param e exception - */ - public static void assertFailureClusterInUse(SliderException e) { - assertExceptionDetails(e, - SliderExitCodes.EXIT_APPLICATION_IN_USE, - ErrorStrings.E_CLUSTER_RUNNING); - } - - protected String buildClustername(String clustername) { - if (SliderUtils.isSet(clustername)) { - return clustername; - } else { - return createClusterName(); - } - } - - /** - * Create the cluster name from the method name and an auto-incrementing - * counter. - * - * @return a cluster name - */ - protected String createClusterName() { - String base = methodName.getMethodName().toLowerCase(Locale.ENGLISH); - if (clusterCount++ > 1) { - return String.format("%s-%d", base, clusterCount); - } - return base; - } - - @Override - public void setup() throws Exception { - super.setup(); - Configuration testConf = getTestConfiguration(); - thawWaitTime = getTimeOptionMillis(testConf, - KEY_TEST_THAW_WAIT_TIME, - thawWaitTime); - freezeWaitTime = getTimeOptionMillis(testConf, - KEY_TEST_FREEZE_WAIT_TIME, - freezeWaitTime); - sliderTestTimeout = getTimeOptionMillis(testConf, - KEY_TEST_TIMEOUT, - sliderTestTimeout); - teardownKillall = - testConf.getBoolean(KEY_TEST_TEARDOWN_KILLALL, - teardownKillall); - - } - - @After - public void teardown() { - describe("teardown"); - stopRunningClusters(); - stopMiniCluster(); - } - - protected void addToTeardown(SliderClient client) { - clustersToTeardown.add(client); - } - - protected void addToTeardown(ServiceLauncher launcher) { - if (launcher != null) { - SliderClient sliderClient = launcher.getService(); - if (sliderClient != null) { - addToTeardown(sliderClient); - } - } - } - - /** - * Kill any java process with the given grep pattern. - * - * @param grepString string to grep for - */ - public int killJavaProcesses(String grepString, int signal) - throws IOException, InterruptedException { - - String[] commandString; - if (!Shell.WINDOWS) { - String killCommand = String.format( - "jps -l| grep %s | awk '{print $1}' | xargs kill %d", grepString, - signal); - LOG.info("Command command = {}", killCommand); - - commandString = new String[]{"bash", "-c", killCommand}; - } else { - // windows - if (!killSupported) { - return -1; - } - - // "jps -l | grep "String" | awk "{print $1}" | xargs -n 1 taskkill /PID" - String killCommand = String.format( - "jps -l | grep %s | gawk '{print $1}' | xargs -n 1 taskkill /f " + - "/PID", grepString); - commandString = new String[]{"CMD", "/C", killCommand}; - } - - Process command = new ProcessBuilder(commandString).start(); - int exitCode = command.waitFor(); - - logStdOutStdErr(command); - return exitCode; - } - - /** - * Kill all processes which match one of the list of grepstrings. - * - * @param greps - * @param signal - */ - public void killJavaProcesses(List greps, int signal) - throws IOException, InterruptedException { - for (String grep : greps) { - killJavaProcesses(grep, signal); - } - } - - protected YarnConfiguration getConfiguration() { - return SLIDER_CONFIG; - } - - /** - * Stop any running cluster that has been added. - */ - public void stopRunningClusters() { - for (SliderClient client : clustersToTeardown) { - maybeStopCluster(client, client.getDeployedClusterName(), - "Teardown at end of test case", true); - } - } - - public void stopMiniCluster() { - Log commonslog = LogFactory.getLog(YarnMiniClusterTestBase.class); - ServiceOperations.stopQuietly(commonslog, miniCluster); - if (hdfsCluster != null) { - hdfsCluster.shutdown(); - } - } - - /** - * Create and start a minicluster. - * - * @param name cluster/test name; if empty one is created from - * the junit method - * @param conf configuration to use - * @param noOfNodeManagers #of NMs - * @param numLocalDirs #of local dirs - * @param numLogDirs #of log dirs - * @param startHDFS create an HDFS mini cluster - * @return the name of the cluster - */ - protected String createMiniCluster(String name, - YarnConfiguration conf, - int noOfNodeManagers, - int numLocalDirs, - int numLogDirs, - boolean startHDFS) throws IOException { - assertNativeLibrariesPresent(); - conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 64); - conf.set(YarnConfiguration.RM_SCHEDULER, FIFO_SCHEDULER); - patchDiskCapacityLimits(conf); - SliderUtils.patchConfiguration(conf); - name = buildClustername(name); - miniCluster = new MiniYARNCluster( - name, - noOfNodeManagers, - numLocalDirs, - numLogDirs); - miniCluster.init(conf); - miniCluster.start(); - // health check - assertMiniClusterDisksHealthy(miniCluster); - if (startHDFS) { - createMiniHDFSCluster(name, conf); - } - return name; - } - - public void patchDiskCapacityLimits(YarnConfiguration conf) { - conf.setFloat( - YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, - 99.0f); - conf.setInt(SliderXmlConfKeys.DFS_NAMENODE_DU_RESERVED_KEY, - 2 * 1024 * 1024); - conf.setBoolean("yarn.nodemanager.disk-health-checker.enable", false); - } - - /** - * Create a mini HDFS cluster and save it to the hdfsClusterField. - * - * @param name - * @param conf - */ - public void createMiniHDFSCluster(String name, YarnConfiguration conf) - throws IOException { - hdfsCluster = buildMiniHDFSCluster(name, conf); - } - - /** - * Launch the client with the specific args against the MiniMR cluster - * launcher i.e. expected to have successfully completed. - * - * @param conf configuration - * @param args arg list - * @return the return code - */ - protected ServiceLauncher launchClientAgainstMiniMR( - Configuration conf, - List args) - throws Throwable { - ServiceLauncher launcher = - launchClientNoExitCodeCheck(conf, args); - int exited = launcher.getServiceExitCode(); - if (exited != 0) { - throw new SliderException(exited, "Launch failed with exit code " + - exited); - } - return launcher; - } - - /** - * Launch the client with the specific args against the MiniMR cluster - * without any checks for exit codes. - * - * @param conf configuration - * @param args arg list - * @return the return code - */ - public ServiceLauncher launchClientNoExitCodeCheck( - Configuration conf, - List args) throws Throwable { - assertNotNull(miniCluster); - return launchClientAgainstRM(getRMAddr(), args, conf); - } - - /** - * Kill all Slider Services. - * - * @param signal - */ - public int killAM(int signal) throws IOException, InterruptedException { - return killJavaProcesses(SliderAppMaster.SERVICE_CLASSNAME_SHORT, signal); - } - - public void logStdOutStdErr(Process p) throws IOException { - try (BufferedReader br = new BufferedReader(new InputStreamReader(p - .getInputStream()))) { - String line = br.readLine(); - while (line != null) { - LOG.info(line); - line = br.readLine(); - } - } - try (BufferedReader br = new BufferedReader(new InputStreamReader(p - .getErrorStream()))) { - String line = br.readLine(); - while (line != null) { - LOG.error(line); - line = br.readLine(); - } - } - } - - /** - * List any java process. - */ - public void lsJavaProcesses() throws InterruptedException, IOException { - Process bash = new ProcessBuilder("jps", "-v").start(); - bash.waitFor(); - logStdOutStdErr(bash); - } - - public YarnConfiguration getTestConfiguration() { - YarnConfiguration conf = getConfiguration(); - conf.addResource(SLIDER_TEST_XML); - return conf; - } - - protected String getRMAddr() { - assertNotNull(miniCluster); - String addr = miniCluster.getConfig().get(YarnConfiguration.RM_ADDRESS); - assertNotNull(addr != null); - assertNotEquals("", addr); - return addr; - } - - /** - * Return the default filesystem, which is HDFS if the miniDFS cluster is - * up, file:// if not. - * - * @return a filesystem string to pass down - */ - protected String getFsDefaultName() { - return buildFsDefaultName(hdfsCluster); - } - - /** - * Create or build a cluster (the action is set by the first verb). - * @param action operation to invoke: SliderActions.ACTION_CREATE or - * SliderActions.ACTION_BUILD - * @param clustername cluster name - * @param extraArgs list of extra args to add to the creation command - * @param deleteExistingData should the data of any existing cluster - * of this name be deleted - * @param blockUntilRunning block until the AM is running - * @return launcher which will have executed the command. - */ - public ServiceLauncher createOrBuildCluster(String action, - String clustername, List extraArgs, boolean deleteExistingData, - boolean blockUntilRunning) throws Throwable { - assertNotNull(clustername); - assertNotNull(miniCluster); - // update action should keep existing data - Configuration config = miniCluster.getConfig(); - if (deleteExistingData && !SliderActions.ACTION_UPDATE.equals(action)) { - FileSystem dfs = FileSystem.get(new URI(getFsDefaultName()), config); - - SliderFileSystem sliderFileSystem = new SliderFileSystem(dfs, config); - Path clusterDir = sliderFileSystem.buildClusterDirPath(clustername); - LOG.info("deleting instance data at {}", clusterDir); - //this is a safety check to stop us doing something stupid like deleting / - assertTrue(clusterDir.toString().contains("/.slider/")); - rigorousDelete(sliderFileSystem, clusterDir, 60000); - } - - - List argsList = new ArrayList<>(); - argsList.addAll(Arrays.asList( - action, clustername, - Arguments.ARG_MANAGER, getRMAddr(), - Arguments.ARG_FILESYSTEM, getFsDefaultName(), - Arguments.ARG_DEBUG)); - - argsList.addAll(getExtraCLIArgs()); - - if (extraArgs != null) { - argsList.addAll(extraArgs); - } - ServiceLauncher launcher = launchClientAgainstMiniMR( - //config includes RM binding info - new YarnConfiguration(config), - //varargs list of command line params - argsList - ); - assertEquals(0, launcher.getServiceExitCode()); - SliderClient client = launcher.getService(); - if (blockUntilRunning) { - client.monitorAppToRunning(new Duration(CLUSTER_GO_LIVE_TIME)); - } - return launcher; - } - - /** - * Delete with some pauses and backoff; designed to handle slow delete - * operation in windows. - */ - public void rigorousDelete( - SliderFileSystem sliderFileSystem, - Path path, long timeout) throws IOException, SliderException { - - if (path.toUri().getScheme() == "file") { - File dir = new File(path.toUri().getPath()); - rigorousDelete(dir, timeout); - } else { - Duration duration = new Duration(timeout); - duration.start(); - FileSystem dfs = sliderFileSystem.getFileSystem(); - boolean deleted = false; - while (!deleted && !duration.getLimitExceeded()) { - dfs.delete(path, true); - deleted = !dfs.exists(path); - if (!deleted) { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - LOG.info("ignoring interrupted sleep"); - } - } - } - } - sliderFileSystem.verifyDirectoryNonexistent(path); - } - - /** - * Delete with some pauses and backoff; designed to handle slow delete - * operation in windows. - * - * @param dir dir to delete - * @param timeout timeout in millis - */ - public void rigorousDelete(File dir, long timeout) throws IOException { - Duration duration = new Duration(timeout); - duration.start(); - boolean deleted = false; - while (!deleted && !duration.getLimitExceeded()) { - FileUtils.deleteQuietly(dir); - deleted = !dir.exists(); - if (!deleted) { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - LOG.info("ignoring interrupted sleep"); - } - } - } - if (!deleted) { - // noisy delete raises an IOE - FileUtils.deleteDirectory(dir); - } - } - - /** - * Add arguments to launch Slider with. - *

- * Extra arguments are added after standard arguments and before roles. - * - * @return additional arguments to launch Slider with - */ - protected List getExtraCLIArgs() { - return new ArrayList<>(); - } - - public String getConfDir() throws FileNotFoundException { - return getResourceConfDirURI(); - } - - /** - * Get the key for the application. - * - * @return - */ - public String getApplicationHomeKey() { - failNotImplemented(); - return null; - } - - /** - * Get the archive path -which defaults to the local one. - * - * @return - */ - public String getArchivePath() { - return getLocalArchive(); - } - - /** - * Get the local archive -the one defined in the test configuration. - * - * @return a possibly null/empty string - */ - public final String getLocalArchive() { - return getTestConfiguration().getTrimmed(getArchiveKey()); - } - - /** - * Get the key for archives in tests. - * - * @return - */ - public String getArchiveKey() { - failNotImplemented(); - return null; - } - - /** - * Merge a k-v pair into a simple k=v string; simple utility. - * - * @param key key - * @param val value - * @return the string to use after a -D option - */ - public String define(String key, String val) { - return String.format("%s=%s", key, val); - } - - public void assumeTestEnabled(boolean flag) { - assume(flag, "test disabled"); - } - - public void assumeArchiveDefined() { - String archive = getArchivePath(); - boolean defined = archive != null && archive != ""; - if (!defined) { - LOG.warn(NO_ARCHIVE_DEFINED + getArchiveKey()); - } - assume(defined, NO_ARCHIVE_DEFINED + getArchiveKey()); - } - - /** - * Assume that application home is defined. This does not check that the - * path is valid -that is expected to be a failure on tests that require - * application home to be set. - */ - public void assumeApplicationHome() { - String applicationHome = getApplicationHome(); - assume(applicationHome != null && applicationHome != "", - "Application home dir option not set " + getApplicationHomeKey()); - } - - public String getApplicationHome() { - return getTestConfiguration().getTrimmed(getApplicationHomeKey()); - } - - /** - * Get the resource configuration dir in the source tree. - * - * @return - */ - public File getResourceConfDir() throws FileNotFoundException { - File f = new File(getTestConfigurationPath()).getAbsoluteFile(); - if (!f.exists()) { - throw new FileNotFoundException( - "Resource configuration directory " + f + " not found"); - } - return f; - } - - public String getTestConfigurationPath() { - failNotImplemented(); - return null; - } - - /** - * Get a URI string to the resource conf dir that is suitable for passing down - * to the AM -and works even when the default FS is hdfs. - */ - public String getResourceConfDirURI() throws FileNotFoundException { - return getResourceConfDir().getAbsoluteFile().toURI().toString(); - } - - /** - * Log an application report. - * - * @param report - */ - public void logReport(ApplicationReport report) { - LOG.info(SliderUtils.reportToString(report)); - } - - /** - * Stop the cluster via the stop action -and wait for - * {@link #CLUSTER_STOP_TIME} for the cluster to stop. If it doesn't - * - * @param sliderClient client - * @param clustername cluster - * @return the exit code - */ - public int clusterActionFreeze(SliderClient sliderClient, String clustername, - String message, boolean force) - throws IOException, YarnException { - LOG.info("Stopping cluster {}: {}", clustername, message); - ActionFreezeArgs freezeArgs = new ActionFreezeArgs(); - freezeArgs.setWaittime(CLUSTER_STOP_TIME); - freezeArgs.message = message; - freezeArgs.force = force; - int exitCode = sliderClient.actionStop(clustername, - freezeArgs); - if (exitCode != 0) { - LOG.warn("Cluster stop failed with error code {}", exitCode); - } - return exitCode; - } - - /** - * Teardown-time cluster termination; will stop the cluster iff the client - * is not null. - * - * @param sliderClient client - * @param clustername name of cluster to teardown - * @return - */ - public int maybeStopCluster( - SliderClient sliderClient, - String clustername, - String message, - boolean force) { - if (sliderClient != null) { - if (SliderUtils.isUnset(clustername)) { - clustername = sliderClient.getDeployedClusterName(); - } - //only stop a cluster that exists - if (SliderUtils.isSet(clustername)) { - try { - clusterActionFreeze(sliderClient, clustername, message, force); - } catch (Exception e) { - LOG.warn("While stopping cluster " + e, e); - } - try { - sliderClient.actionDestroy(clustername); - } catch (Exception e) { - LOG.warn("While destroying cluster " + e, e); - } - } - } - return 0; - } - - public String roleMapToString(Map roles) { - StringBuilder builder = new StringBuilder(); - for (Entry entry : roles.entrySet()) { - builder.append(entry.getKey()); - builder.append("->"); - builder.append(entry.getValue()); - builder.append(" "); - } - return builder.toString(); - } - - /** - * Turn on test runs against a copy of the archive that is - * uploaded to HDFS -this method copies up the - * archive then switches the tests into archive mode. - */ - public void enableTestRunAgainstUploadedArchive() throws IOException { - Path remotePath = copyLocalArchiveToHDFS(getLocalArchive()); - // image mode - switchToRemoteImageDeploy(remotePath); - } - - /** - * Switch to deploying a remote image. - * - * @param remotePath the remote path to use - */ - public void switchToRemoteImageDeploy(Path remotePath) { - switchToImageDeploy = true; - imageIsRemote = true; - remoteImageURI = remotePath.toUri(); - } - - /** - * Copy a local archive to HDFS. - * - * @param localArchive local archive - * @return the path of the uploaded image - */ - public Path copyLocalArchiveToHDFS(String localArchive) throws IOException { - assertNotNull(localArchive); - File localArchiveFile = new File(localArchive); - assertTrue(localArchiveFile.exists()); - assertNotNull(hdfsCluster); - Path remoteUnresolvedArchive = new Path(localArchiveFile.getName()); - assertTrue(FileUtil.copy( - localArchiveFile, - hdfsCluster.getFileSystem(), - remoteUnresolvedArchive, - false, - getTestConfiguration())); - Path remotePath = hdfsCluster.getFileSystem().resolvePath( - remoteUnresolvedArchive); - return remotePath; - } - - /** - * Create a SliderFileSystem instance bonded to the running FS. - * The YARN cluster must be up and running already - * - * @return - */ - public SliderFileSystem createSliderFileSystem() - throws URISyntaxException, IOException { - FileSystem dfs = - FileSystem.get(new URI(getFsDefaultName()), getConfiguration()); - SliderFileSystem hfs = new SliderFileSystem(dfs, getConfiguration()); - return hfs; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/YarnZKMiniClusterTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/YarnZKMiniClusterTestBase.java deleted file mode 100644 index cf9e616..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/utils/YarnZKMiniClusterTestBase.java +++ /dev/null @@ -1,176 +0,0 @@ -/* - * 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.slider.utils; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.registry.client.api.RegistryConstants; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.slider.core.zk.BlockingZKWatcher; -import org.apache.slider.core.zk.ZKIntegration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; - -import static org.apache.slider.utils.KeysForTests.USERNAME; - -/** - * Base class for mini cluster tests that use Zookeeper. - */ -public abstract class YarnZKMiniClusterTestBase extends - YarnMiniClusterTestBase { - private static final Logger LOG = - LoggerFactory.getLogger(YarnZKMiniClusterTestBase.class); - - private MicroZKCluster microZKCluster; - - public void stopMiniCluster() { - super.stopMiniCluster(); - IOUtils.closeStream(microZKCluster); - } - - public ZKIntegration createZKIntegrationInstance(String zkQuorum, - String clusterName, - boolean createClusterPath, - boolean canBeReadOnly, - int timeout) throws IOException, InterruptedException { - int sessionTimeout = ZKIntegration.SESSION_TIMEOUT; - - BlockingZKWatcher watcher = new BlockingZKWatcher(); - ZKIntegration zki = ZKIntegration.newInstance(zkQuorum, - USERNAME, - clusterName, - createClusterPath, - canBeReadOnly, - watcher, - sessionTimeout); - boolean fromCache = zki.init(); - //here the callback may or may not have occurred. - //optionally wait for it - if (timeout > 0 && !fromCache) { - watcher.waitForZKConnection(timeout); - } - //if we get here, the binding worked - LOG.info("Connected: {}", zki); - return zki; - } - - /** - * Wait for a flag to go true. - * @param connectedFlag - */ - public void waitForZKConnection(AtomicBoolean connectedFlag, int timeout) - throws InterruptedException { - synchronized (connectedFlag) { - if (!connectedFlag.get()) { - LOG.info("waiting for ZK event"); - //wait a bit - connectedFlag.wait(timeout); - } - } - assertTrue(connectedFlag.get()); - } - - /** - * Create and start a minicluster with ZK. - * @param name cluster/test name - * @param conf configuration to use - * @param noOfNodeManagers #of NMs - * @param numLocalDirs #of local dirs - * @param numLogDirs #of log dirs - * @param startZK create a ZK micro cluster *THIS IS IGNORED* - * @param startHDFS create an HDFS mini cluster - */ - protected String createMiniCluster(String name, - YarnConfiguration conf, - int noOfNodeManagers, - int numLocalDirs, - int numLogDirs, - boolean startZK, - boolean startHDFS) throws IOException { - name = buildClustername(name); - createMicroZKCluster("-" + name, conf); - conf.setBoolean(RegistryConstants.KEY_REGISTRY_ENABLED, true); - conf.set(RegistryConstants.KEY_REGISTRY_ZK_QUORUM, getZKBinding()); - //now create the cluster - name = super.createMiniCluster(name, conf, noOfNodeManagers, - numLocalDirs, numLogDirs, startHDFS); - - return name; - } - - /** - * Create and start a minicluster. - * @param name cluster/test name - * @param conf configuration to use - * @param noOfNodeManagers #of NMs - * @param startZK create a ZK micro cluster - */ - protected String createMiniCluster(String name, - YarnConfiguration conf, - int noOfNodeManagers, - boolean startZK) throws IOException { - return createMiniCluster(name, conf, noOfNodeManagers, 1, 1, startZK, - false); - } - - /** - * Create and start a minicluster with the name from the test method. - * @param conf configuration to use - * @param noOfNodeManagers #of NMs - * @param startZK create a ZK micro cluster - */ - protected String createMiniCluster(YarnConfiguration conf, - int noOfNodeManagers, - boolean startZK) throws IOException { - return createMiniCluster("", conf, noOfNodeManagers, 1, 1, startZK, - false); - } - - public void createMicroZKCluster(String name, Configuration conf) { - microZKCluster = new MicroZKCluster(new Configuration(conf)); - microZKCluster.createCluster(name); - } - - public void assertHasZKCluster() { - assertNotNull(microZKCluster); - } - - public String getZKBinding() { - if (microZKCluster == null) { - return "localhost:1"; - } else { - return microZKCluster.getZkBindingString(); - } - } - - /** - * CLI args include all the ZK bindings needed. - * @return - */ - protected List getExtraCLIArgs() { - return Arrays.asList( - "-D", define(RegistryConstants.KEY_REGISTRY_ZK_QUORUM, getZKBinding()) - ); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/hadoop/yarn/service/conf/examples/app.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/hadoop/yarn/service/conf/examples/app.json index b1d73c5..a163b33 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/hadoop/yarn/service/conf/examples/app.json +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/hadoop/yarn/service/conf/examples/app.json @@ -6,7 +6,7 @@ "properties": { "g1": "a", "g2": "b", - "internal.chaos.monkey.interval.seconds": "60" + "yarn.service.failure-count-reset.window": "60" } }, "resource": { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/server/appmaster/web/rest/registry/sample.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/server/appmaster/web/rest/registry/sample.json deleted file mode 100644 index bc6429c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/server/appmaster/web/rest/registry/sample.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "nodes": ["/users/example/services/org-apache-slider/test-registry-rest-resources/components"], "service": { - "description": "Slider Application Master", - "yarn:id": "application_1411664296263_0001", - "yarn:persistence": 1, - "external": [], - "internal": [] -} -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/server/avro/history-v01-3-role.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/server/avro/history-v01-3-role.json deleted file mode 100644 index ceab0a5..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/server/avro/history-v01-3-role.json +++ /dev/null @@ -1,6 +0,0 @@ -{"entry":{"org.apache.slider.server.avro.RoleHistoryHeader":{"version":1,"saved":1415296260647,"savedx":"149863b1a27","savedate":"6 Nov 2014 17:51:00 GMT","roles":3}}} -{"entry":{"org.apache.slider.server.avro.NodeEntryRecord":{"host":"192.168.1.85","role":1,"active":false,"last_used":0}}} -{"entry":{"org.apache.slider.server.avro.NodeEntryRecord":{"host":"192.168.1.85","role":2,"active":false,"last_used":0}}} -{"entry":{"org.apache.slider.server.avro.NodeEntryRecord":{"host":"192.168.1.85","role":0,"active":false,"last_used":0}}} -{"entry":{"org.apache.slider.server.avro.NodeEntryRecord":{"host":"192.168.1.86","role":2,"active":true,"last_used":0}}} -{"entry":{"org.apache.slider.server.avro.RoleHistoryFooter":{"count":4}}} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/server/avro/history-v01-6-role.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/server/avro/history-v01-6-role.json deleted file mode 100644 index f1c53d5..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/server/avro/history-v01-6-role.json +++ /dev/null @@ -1,8 +0,0 @@ -{"entry":{"org.apache.slider.server.avro.RoleHistoryHeader":{"version":1,"saved":1415296260647,"savedx":"149863b1a27","savedate":"6 Nov 2014 17:51:00 GMT","roles":6}}} -{"entry":{"org.apache.slider.server.avro.NodeEntryRecord":{"host":"192.168.1.85","role":1,"active":false,"last_used":0}}} -{"entry":{"org.apache.slider.server.avro.NodeEntryRecord":{"host":"192.168.1.85","role":2,"active":false,"last_used":0}}} -{"entry":{"org.apache.slider.server.avro.NodeEntryRecord":{"host":"192.168.1.85","role":0,"active":false,"last_used":0}}} -{"entry":{"org.apache.slider.server.avro.NodeEntryRecord":{"host":"192.168.1.86","role":4,"active":true,"last_used":0}}} -{"entry":{"org.apache.slider.server.avro.NodeEntryRecord":{"host":"192.168.1.86","role":5,"active":true,"last_used":0}}} -{"entry":{"org.apache.slider.server.avro.NodeEntryRecord":{"host":"192.168.1.86","role":6,"active":true,"last_used":0}}} -{"entry":{"org.apache.slider.server.avro.RoleHistoryFooter":{"count":6}}} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/server/avro/history_v01b_1_role.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/server/avro/history_v01b_1_role.json deleted file mode 100644 index 67d644f..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/server/avro/history_v01b_1_role.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "entry": { - "org.apache.slider.server.avro.RoleHistoryHeader": { - "version": 1, - "saved": 1450435691617, - "savedx": "151b4b44461", - "savedate": "18 Dec 2015 10:48:11 GMT", - "roles": 2 - } - } -} -{ - "entry": { - "org.apache.slider.server.avro.RoleHistoryMapping": { - "rolemap": { - "echo": 1, - "slider-appmaster": 0 - } - } - } -} -{ - "entry": { - "org.apache.slider.server.avro.NodeEntryRecord": { - "host": "192.168.56.1", - "role": 1, - "active": true, - "last_used": 0 - } - } -} -{ - "entry": { - "org.apache.slider.server.avro.RoleHistoryFooter": { - "count": 1 - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-services.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-services.js index fb01138..0f74f2f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-services.js +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-services.js @@ -23,7 +23,7 @@ export default AbstractRoute.extend({ model() { return Ember.RSVP.hash({ apps: this.store.query('yarn-app', { - applicationTypes: "org-apache-slider" + applicationTypes: "yarn-native-service" }), }); },