Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 313EA200D16 for ; Tue, 26 Sep 2017 01:38:26 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 2F84B1609EE; Mon, 25 Sep 2017 23:38:26 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id C253E1609E9 for ; Tue, 26 Sep 2017 01:38:23 +0200 (CEST) Received: (qmail 1277 invoked by uid 500); 25 Sep 2017 23:38:05 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 98872 invoked by uid 99); 25 Sep 2017 23:38:03 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 25 Sep 2017 23:38:03 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id A7293F5B27; Mon, 25 Sep 2017 23:38:01 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jianhe@apache.org To: common-commits@hadoop.apache.org Date: Mon, 25 Sep 2017 23:38:54 -0000 Message-Id: <5ddc708f6101455598de745e44ceaa87@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [55/86] [abbrv] hadoop git commit: YARN-7091. Rename application to service in yarn-native-services. Contributed by Jian He archived-at: Mon, 25 Sep 2017 23:38:26 -0000 http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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 deleted file mode 100644 index 1049698..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/client/ServiceClient.java +++ /dev/null @@ -1,872 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.service.client; - -import org.apache.commons.lang.StringUtils; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.retry.RetryNTimes; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.registry.client.api.RegistryConstants; -import org.apache.hadoop.registry.client.api.RegistryOperations; -import org.apache.hadoop.registry.client.api.RegistryOperationsFactory; -import org.apache.hadoop.registry.client.binding.RegistryUtils; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.service.CompositeService; -import org.apache.hadoop.util.VersionInfo; -import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; -import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; -import org.apache.hadoop.yarn.api.records.ApplicationTimeout; -import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.LocalResource; -import org.apache.hadoop.yarn.api.records.LocalResourceType; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.client.api.YarnClient; -import org.apache.hadoop.yarn.client.api.YarnClientApplication; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.ipc.YarnRPC; -import org.apache.hadoop.yarn.proto.ClientAMProtocol.ComponentCountProto; -import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto; -import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto; -import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto; -import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopRequestProto; -import org.apache.hadoop.yarn.service.ClientAMProtocol; -import org.apache.hadoop.yarn.service.ServiceMaster; -import org.apache.hadoop.yarn.service.api.records.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.YarnServiceConstants; -import org.apache.hadoop.yarn.service.conf.YarnServiceConf; -import org.apache.hadoop.yarn.service.provider.AbstractClientProvider; -import org.apache.hadoop.yarn.service.provider.ProviderUtils; -import org.apache.hadoop.yarn.service.utils.ServiceApiUtil; -import org.apache.hadoop.yarn.service.utils.ServiceRegistryUtils; -import org.apache.hadoop.yarn.service.utils.SliderFileSystem; -import org.apache.hadoop.yarn.service.utils.SliderUtils; -import org.apache.hadoop.yarn.util.Records; -import org.apache.hadoop.yarn.util.Times; -import org.apache.hadoop.yarn.service.exceptions.BadClusterStateException; -import org.apache.hadoop.yarn.service.exceptions.BadConfigException; -import org.apache.hadoop.yarn.service.exceptions.SliderException; -import org.apache.hadoop.yarn.service.exceptions.UsageException; -import org.apache.hadoop.yarn.service.containerlaunch.ClasspathConstructor; -import org.apache.hadoop.yarn.service.containerlaunch.JavaCommandLineBuilder; -import org.apache.hadoop.yarn.service.utils.ZookeeperUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.text.MessageFormat; -import java.util.Collections; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; - -import static org.apache.hadoop.yarn.api.records.YarnApplicationState.*; -import static org.apache.hadoop.yarn.service.client.params.SliderActions.ACTION_CREATE; -import static org.apache.hadoop.yarn.service.client.params.SliderActions.ACTION_FLEX; -import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.YARN_QUEUE; -import static org.apache.hadoop.yarn.service.utils.SliderUtils.*; - -@InterfaceAudience.Public -@InterfaceStability.Unstable -public class ServiceClient extends CompositeService - implements SliderExitCodes, YarnServiceConstants { - private static final Logger LOG = - LoggerFactory.getLogger(ServiceClient.class); - private SliderFileSystem fs; - 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; - - private static EnumSet terminatedStates = - EnumSet.of(FINISHED, FAILED, KILLED); - private static EnumSet liveStates = - EnumSet.of(NEW, NEW_SAVING, SUBMITTED, RUNNING); - - public ServiceClient() { - super(ServiceClient.class.getName()); - } - - @Override protected void serviceInit(Configuration configuration) - throws Exception { - fs = new SliderFileSystem(configuration); - yarnClient = YarnClient.createYarnClient(); - rpc = YarnRPC.create(configuration); - addService(yarnClient); - super.serviceInit(configuration); - } - - @Override - protected void serviceStop() throws Exception { - if (registryClient != null) { - registryClient.stop(); - } - super.serviceStop(); - } - - private Application loadAppJsonFromLocalFS( - AbstractClusterBuildingActionArgs args) throws IOException { - File file = args.getAppDef(); - Path filePath = new Path(file.getAbsolutePath()); - LOG.info("Loading app json from: " + filePath); - Application application = ServiceApiUtil.jsonSerDeser - .load(FileSystem.getLocal(getConfig()), filePath); - if (args.lifetime > 0) { - application.setLifetime(args.lifetime); - } - application.setName(args.getClusterName()); - return application; - } - - public int actionBuild(AbstractClusterBuildingActionArgs args) - throws IOException, YarnException { - return actionBuild(loadAppJsonFromLocalFS(args)); - } - - public int actionBuild(Application application) - throws YarnException, IOException { - Path appDir = checkAppNotExistOnHdfs(application); - ServiceApiUtil.validateAndResolveApplication(application, fs, getConfig()); - createDirAndPersistApp(appDir, application); - return EXIT_SUCCESS; - } - - public int actionCreate(AbstractClusterBuildingActionArgs args) - throws IOException, YarnException { - actionCreate(loadAppJsonFromLocalFS(args)); - return EXIT_SUCCESS; - } - - public ApplicationId actionCreate(Application application) - throws IOException, YarnException { - String appName = application.getName(); - validateClusterName(appName); - ServiceApiUtil.validateAndResolveApplication(application, fs, getConfig()); - verifyNoLiveAppInRM(appName, "create"); - Path appDir = checkAppNotExistOnHdfs(application); - - // Write the definition first and then submit - AM will read the definition - createDirAndPersistApp(appDir, application); - ApplicationId appId = submitApp(application); - cachedAppIds.put(appName, appId); - application.setId(appId.toString()); - // update app definition with appId - persistAppDef(appDir, application); - return appId; - } - - // Called by ServiceCLI - protected int actionFlexByCLI(ClientArgs args) - throws YarnException, IOException { - ActionFlexArgs flexArgs = args.getActionFlexArgs(); - Map componentCounts = - new HashMap<>(flexArgs.getComponentMap().size()); - Application persistedApp = - ServiceApiUtil.loadApplication(fs, flexArgs.getClusterName()); - if (!StringUtils.isEmpty(persistedApp.getId())) { - cachedAppIds.put(persistedApp.getName(), - ApplicationId.fromString(persistedApp.getId())); - } - for (Map.Entry entry : flexArgs.getComponentMap() - .entrySet()) { - String compName = entry.getKey(); - ServiceApiUtil.validateCompName(compName); - Component component = persistedApp.getComponent(compName); - if (component == null) { - throw new IllegalArgumentException(entry.getKey() + " does not exist !"); - } - long numberOfContainers = - parseNumberOfContainers(component, entry.getValue()); - componentCounts.put(compName, numberOfContainers); - } - // throw usage exception if no changes proposed - if (componentCounts.size() == 0) { - actionHelp(ACTION_FLEX, args); - } - flexComponents(args.getClusterName(), componentCounts, persistedApp); - return EXIT_SUCCESS; - } - - // Parse the number of containers requested by user, e.g. - // +5 means add 5 additional containers - // -5 means reduce 5 containers, if it goes to negative, sets it to 0 - // 5 means sets it to 5 containers. - private long parseNumberOfContainers(Component component, String newNumber) { - - long orig = component.getNumberOfContainers(); - if (newNumber.startsWith("+")) { - return orig + Long.parseLong(newNumber.substring(1)); - } else if (newNumber.startsWith("-")) { - long ret = orig - Long.parseLong(newNumber.substring(1)); - if (ret < 0) { - LOG.warn(MessageFormat.format( - "[COMPONENT {}]: component count goes to negative ({}{} = {}), reset it to 0.", - component.getName(), orig, newNumber, ret)); - ret = 0; - } - return ret; - } else { - return Long.parseLong(newNumber); - } - } - - // Called by Rest Service - public Map flexByRestService(String appName, - Map componentCounts) throws YarnException, IOException { - // load app definition - Application persistedApp = ServiceApiUtil.loadApplication(fs, appName); - cachedAppIds.put(persistedApp.getName(), - ApplicationId.fromString(persistedApp.getId())); - return flexComponents(appName, componentCounts, persistedApp); - } - - private Map flexComponents(String appName, - Map componentCounts, Application persistedApp) - throws YarnException, IOException { - validateClusterName(appName); - - Map original = new HashMap<>(componentCounts.size()); - - ComponentCountProto.Builder countBuilder = ComponentCountProto.newBuilder(); - FlexComponentsRequestProto.Builder requestBuilder = - FlexComponentsRequestProto.newBuilder(); - - for (Component persistedComp : persistedApp.getComponents()) { - String name = persistedComp.getName(); - if (componentCounts.containsKey(persistedComp.getName())) { - original.put(name, persistedComp.getNumberOfContainers()); - persistedComp.setNumberOfContainers(componentCounts.get(name)); - - // build the request - countBuilder.setName(persistedComp.getName()) - .setNumberOfContainers(persistedComp.getNumberOfContainers()); - requestBuilder.addComponents(countBuilder.build()); - } - } - if (original.size() < componentCounts.size()) { - componentCounts.keySet().removeAll(original.keySet()); - throw new YarnException("Components " + componentCounts.keySet() - + " do not exist in app definition."); - } - ServiceApiUtil.jsonSerDeser - .save(fs.getFileSystem(), ServiceApiUtil.getAppJsonPath(fs, appName), - persistedApp, true); - 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 {}", - entry.getKey(), entry.getValue(), - componentCounts.get(entry.getKey())); - } - return original; - } - - public int actionStop(String appName, boolean waitForAppStopped) - throws YarnException, IOException { - validateClusterName(appName); - getAppId(appName); - ApplicationId currentAppId = cachedAppIds.get(appName); - ApplicationReport report = yarnClient.getApplicationReport(currentAppId); - if (terminatedStates.contains(report.getYarnApplicationState())) { - LOG.info("Application {} is already in a terminated state {}", appName, - report.getYarnApplicationState()); - return EXIT_SUCCESS; - } - LOG.info("Stopping application {}, with appId = {}", appName, currentAppId); - try { - 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(2000); - report = yarnClient.getApplicationReport(currentAppId); - if (terminatedStates.contains(report.getYarnApplicationState())) { - LOG.info("Application " + appName + " is stopped."); - break; - } - // Forcefully kill after 10 seconds. - if ((System.currentTimeMillis() - startTime) > 10000) { - LOG.info("Stop operation timeout stopping, forcefully kill the app " - + appName); - yarnClient.killApplication(currentAppId, - "Forcefully kill the app by user"); - 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(currentAppId, "Forcefully kill the app"); - } - return EXIT_SUCCESS; - } - - public int actionDestroy(String appName) throws Exception { - validateClusterName(appName); - verifyNoLiveAppInRM(appName, "Destroy"); - Path appDir = fs.buildClusterDirPath(appName); - 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 + ": " - + appDir); - } else { - String message = - "Failed to delete application + " + appName + " at: " + appDir; - LOG.info(message); - throw new YarnException(message); - } - } - deleteZKNode(appName); - String registryPath = ServiceRegistryUtils.registryPathForInstance(appName); - try { - getRegistryClient().delete(registryPath, true); - } catch (IOException e) { - LOG.warn("Error deleting registry entry {}", registryPath, e); - } - LOG.info("Destroyed cluster {}", appName); - return EXIT_SUCCESS; - } - - private synchronized RegistryOperations getRegistryClient() - throws SliderException, IOException { - - if (registryClient == null) { - registryClient = - RegistryOperationsFactory.createInstance("ServiceClient", getConfig()); - registryClient.init(getConfig()); - registryClient.start(); - } - return registryClient; - } - - private void deleteZKNode(String clusterName) throws Exception { - CuratorFramework curatorFramework = getCuratorClient(); - String user = RegistryUtils.currentUser(); - String zkPath = ServiceRegistryUtils.mkClusterPath(user, clusterName); - if (curatorFramework.checkExists().forPath(zkPath) != null) { - curatorFramework.delete().deletingChildrenIfNeeded().forPath(zkPath); - LOG.info("Deleted zookeeper path: " + zkPath); - } - } - - private synchronized CuratorFramework getCuratorClient() - throws BadConfigException { - String registryQuorum = - getConfig().get(RegistryConstants.KEY_REGISTRY_ZK_QUORUM); - - // though if neither is set: trouble - if (SliderUtils.isUnset(registryQuorum)) { - throw new BadConfigException( - "No Zookeeper quorum provided in the" + " configuration property " - + RegistryConstants.KEY_REGISTRY_ZK_QUORUM); - } - ZookeeperUtils.splitToHostsAndPortsStrictly(registryQuorum); - - if (curatorClient == null) { - curatorClient = - CuratorFrameworkFactory.builder().connectString(registryQuorum) - .sessionTimeoutMs(10000).retryPolicy(new RetryNTimes(5, 2000)) - .build(); - curatorClient.start(); - } - return curatorClient; - } - - private int actionHelp(String actionName, CommonArgs args) - throws YarnException, IOException { - throw new UsageException(CommonArgs.usage(args, actionName)); - } - - private void verifyNoLiveAppInRM(String appname, String action) - throws IOException, YarnException { - Set types = new HashSet<>(1); - types.add(YarnServiceConstants.APP_TYPE); - Set tags = null; - if (appname != null) { - tags = Collections.singleton(SliderUtils.createNameTag(appname)); - } - GetApplicationsRequest request = GetApplicationsRequest.newInstance(); - request.setApplicationTypes(types); - request.setApplicationTags(tags); - request.setApplicationStates(liveStates); - List reports = yarnClient.getApplications(request); - if (!reports.isEmpty()) { - throw new YarnException( - "Failed to " + action + " application, as " + appname - + " already exists."); - } - } - - private ApplicationId submitApp(Application app) - throws IOException, YarnException { - String appName = app.getName(); - Configuration conf = getConfig(); - Path appRootDir = fs.buildClusterDirPath(app.getName()); - - YarnClientApplication yarnApp = yarnClient.createApplication(); - ApplicationSubmissionContext submissionContext = - yarnApp.getApplicationSubmissionContext(); - ServiceApiUtil.validateCompResourceSize( - yarnApp.getNewApplicationResponse().getMaximumResourceCapability(), - app); - - submissionContext.setKeepContainersAcrossApplicationAttempts(true); - if (app.getLifetime() > 0) { - Map appTimeout = new HashMap<>(); - appTimeout.put(ApplicationTimeoutType.LIFETIME, app.getLifetime()); - submissionContext.setApplicationTimeouts(appTimeout); - } - submissionContext.setMaxAppAttempts(conf.getInt( - YarnServiceConf.AM_RESTART_MAX, 2)); - - Map localResources = new HashMap<>(); - - // copy local slideram-log4j.properties to hdfs and add to localResources - boolean hasAMLog4j = - addAMLog4jResource(appName, conf, localResources); - // copy jars to hdfs and add to localResources - addJarResource(appName, localResources); - // add keytab if in secure env - addKeytabResourceIfSecure(fs, localResources, conf, appName); - if (LOG.isDebugEnabled()) { - printLocalResources(localResources); - } - Map env = addAMEnv(conf); - - // create AM CLI - String cmdStr = - buildCommandLine(appName, conf, appRootDir, hasAMLog4j); - submissionContext.setResource(Resource.newInstance(YarnServiceConf - .getLong(YarnServiceConf.AM_RESOURCE_MEM, YarnServiceConf.DEFAULT_KEY_AM_RESOURCE_MEM, - app.getConfiguration(), conf), 1)); - String queue = app.getQueue(); - if (StringUtils.isEmpty(queue)) { - queue = conf.get(YARN_QUEUE, "default"); - } - submissionContext.setQueue(queue); - submissionContext.setApplicationName(appName); - submissionContext.setApplicationType(YarnServiceConstants.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); - submissionContext.setAMContainerSpec(amLaunchContext); - yarnClient.submitApplication(submissionContext); - return submissionContext.getApplicationId(); - } - - private void printLocalResources(Map map) { - LOG.debug("Added LocalResource for localization: "); - StringBuilder builder = new StringBuilder(); - for (Map.Entry entry : map.entrySet()) { - builder.append(entry.getKey()).append(" -> ") - .append(entry.getValue().getResource().getFile()) - .append(System.lineSeparator()); - } - LOG.debug(builder.toString()); - } - - private String buildCommandLine(String 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, YARN_SERVICE_LOG4J_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")); - // pass the registry binding - CLI.addConfOptionToCLI(conf, RegistryConstants.KEY_REGISTRY_ZK_ROOT, - RegistryConstants.DEFAULT_ZK_REGISTRY_ROOT); - CLI.addMandatoryConfOption(conf, RegistryConstants.KEY_REGISTRY_ZK_QUORUM); - - // write out the path output - CLI.addOutAndErrFiles(STDOUT_AM, STDERR_AM); - String cmdStr = CLI.build(); - LOG.info("AM launch command: {}", cmdStr); - return cmdStr; - } - - private Map addAMEnv(Configuration conf) throws IOException { - Map env = new HashMap<>(); - ClasspathConstructor classpath = - buildClasspath(YarnServiceConstants.SUBMITTED_CONF_DIR, "lib", fs, getConfig() - .getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, false)); - env.put("CLASSPATH", classpath.buildClasspath()); - env.put("LANG", "en_US.UTF-8"); - env.put("LC_ALL", "en_US.UTF-8"); - env.put("LANGUAGE", "en_US.UTF-8"); - String jaas = System.getenv("HADOOP_JAAS_DEBUG"); - if (jaas != null) { - env.put("HADOOP_JAAS_DEBUG", jaas); - } - if (!UserGroupInformation.isSecurityEnabled()) { - String userName = UserGroupInformation.getCurrentUser().getUserName(); - LOG.info("Run as user " + userName); - // HADOOP_USER_NAME env is used by UserGroupInformation when log in - // This env makes AM run as this user - env.put("HADOOP_USER_NAME", userName); - } - LOG.info("AM env: \n{}", stringifyMap(env)); - return env; - } - - protected Path addJarResource(String appName, - Map localResources) - throws IOException, SliderException { - Path libPath = fs.buildClusterDirPath(appName); - ProviderUtils - .addProviderJar(localResources, ServiceMaster.class, SERVICE_CORE_JAR, fs, - libPath, "lib", false); - Path dependencyLibTarGzip = fs.getDependencyTarGzip(); - if (fs.isFile(dependencyLibTarGzip)) { - LOG.info("Loading lib tar from " + fs.getFileSystem().getScheme() + ":/" - + dependencyLibTarGzip); - SliderUtils.putAmTarGzipAndUpdate(localResources, fs); - } else { - String[] libs = SliderUtils.getLibDirs(); - for (String libDirProp : libs) { - ProviderUtils.addAllDependencyJars(localResources, fs, libPath, "lib", - libDirProp); - } - } - return libPath; - } - - private boolean addAMLog4jResource(String appName, Configuration conf, - Map localResources) - throws IOException, BadClusterStateException { - boolean hasAMLog4j = false; - String hadoopConfDir = - System.getenv(ApplicationConstants.Environment.HADOOP_CONF_DIR.name()); - if (hadoopConfDir != null) { - File localFile = - new File(hadoopConfDir, YarnServiceConstants.YARN_SERVICE_LOG4J_FILENAME); - if (localFile.exists()) { - Path localFilePath = createLocalPath(localFile); - Path appDirPath = fs.buildClusterDirPath(appName); - Path remoteConfPath = - new Path(appDirPath, YarnServiceConstants.SUBMITTED_CONF_DIR); - Path remoteFilePath = - new Path(remoteConfPath, YarnServiceConstants.YARN_SERVICE_LOG4J_FILENAME); - copy(conf, localFilePath, remoteFilePath); - LocalResource localResource = - fs.createAmResource(remoteConfPath, LocalResourceType.FILE); - localResources.put(localFilePath.getName(), localResource); - hasAMLog4j = true; - } else { - LOG.warn("AM log4j property file doesn't exist: " + localFile); - } - } - return hasAMLog4j; - } - - public int actionStart(String appName) throws YarnException, IOException { - validateClusterName(appName); - Path appDir = checkAppExistOnHdfs(appName); - Application application = ServiceApiUtil.loadApplication(fs, appName); - ServiceApiUtil.validateAndResolveApplication(application, fs, getConfig()); - // see if it is actually running and bail out; - verifyNoLiveAppInRM(appName, "thaw"); - ApplicationId appId = submitApp(application); - application.setId(appId.toString()); - // write app definition on to hdfs - createDirAndPersistApp(appDir, application); - return 0; - } - - private Path checkAppNotExistOnHdfs(Application application) - throws IOException, SliderException { - Path appDir = fs.buildClusterDirPath(application.getName()); - fs.verifyDirectoryNonexistent( - new Path(appDir, application.getName() + ".json")); - return appDir; - } - - private Path checkAppExistOnHdfs(String appName) - throws IOException, SliderException { - Path appDir = fs.buildClusterDirPath(appName); - fs.verifyPathExists(new Path(appDir, appName + ".json")); - return appDir; - } - - private void createDirAndPersistApp(Path appDir, Application application) - throws IOException, SliderException { - FsPermission appDirPermission = new FsPermission("750"); - fs.createWithPermissions(appDir, appDirPermission); - persistAppDef(appDir, application); - } - - private void persistAppDef(Path appDir, Application application) - throws IOException { - Path appJson = new Path(appDir, application.getName() + ".json"); - ServiceApiUtil.jsonSerDeser - .save(fs.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(YarnServiceConf.KEY_AM_KEYTAB_LOCAL_PATH); - if (StringUtils.isEmpty(keytabPreInstalledOnHost)) { - String amKeytabName = - conf.get(YarnServiceConf.KEY_AM_LOGIN_KEYTAB_NAME); - String keytabDir = conf.get(YarnServiceConf.KEY_HDFS_KEYTAB_DIR); - Path keytabPath = - fileSystem.buildKeytabPath(keytabDir, amKeytabName, appName); - if (fileSystem.getFileSystem().exists(keytabPath)) { - LocalResource keytabRes = - fileSystem.createAmResource(keytabPath, LocalResourceType.FILE); - localResource - .put(YarnServiceConstants.KEYTAB_DIR + "/" + amKeytabName, keytabRes); - LOG.info("Adding AM keytab on hdfs: " + keytabPath); - } else { - LOG.warn("No keytab file was found at {}.", keytabPath); - if (conf.getBoolean(YarnServiceConf.KEY_AM_LOGIN_KEYTAB_REQUIRED, false)) { - throw new BadConfigException("No keytab file was found at %s.", - keytabPath); - } else { - LOG.warn("The AM will be " - + "started without a kerberos authenticated identity. " - + "The application is therefore not guaranteed to remain " - + "operational beyond 24 hours."); - } - } - } - } - - public String updateLifetime(String appName, long lifetime) - throws YarnException, IOException { - getAppId(appName); - ApplicationId currentAppId = cachedAppIds.get(appName); - ApplicationReport report = yarnClient.getApplicationReport(currentAppId); - 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; - } - - public Application getStatus(String appName) - throws IOException, YarnException { - 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() { - return this.yarnClient; - } - - public int actionDependency(ActionDependencyArgs args) - throws IOException, YarnException { - String currentUser = RegistryUtils.currentUser(); - LOG.info("Running command as user {}", currentUser); - - Path dependencyLibTarGzip = fs.getDependencyTarGzip(); - - // Check if dependency has already been uploaded, in which case log - // appropriately and exit success (unless overwrite has been requested) - if (fs.isFile(dependencyLibTarGzip) && !args.overwrite) { - System.out.println(String.format( - "Dependency libs are already uploaded to %s. Use %s " - + "if you want to re-upload", dependencyLibTarGzip.toUri(), - Arguments.ARG_OVERWRITE)); - return EXIT_SUCCESS; - } - - String[] libDirs = SliderUtils.getLibDirs(); - if (libDirs.length > 0) { - File tempLibTarGzipFile = File.createTempFile( - YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_NAME + "_", - YarnServiceConstants.DEPENDENCY_TAR_GZ_FILE_EXT); - // copy all jars - tarGzipFolder(libDirs, tempLibTarGzipFile, createJarFilter()); - - LOG.info("Version Info: " + VersionInfo.getBuildVersion()); - fs.copyLocalFileToHdfs(tempLibTarGzipFile, dependencyLibTarGzip, - new FsPermission(YarnServiceConstants.DEPENDENCY_DIR_PERMISSIONS)); - return EXIT_SUCCESS; - } else { - return EXIT_FALSE; - } - } - - // 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 = getAppId(appName); - - 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; - } - } - - 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 getAppId(String appName) - throws IOException, YarnException { - if (cachedAppIds.containsKey(appName)) { - return cachedAppIds.get(appName); - } - Application persistedApp = ServiceApiUtil.loadApplication(fs, appName); - if (persistedApp == null) { - throw new YarnException("Application " + appName - + " doesn't exist on hdfs. Please check if the app exists in RM"); - } - ApplicationId currentAppId = ApplicationId.fromString(persistedApp.getId()); - cachedAppIds.put(appName, currentAppId); - return currentAppId; - } -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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 deleted file mode 100644 index ea3bb0a..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/client/params/AbstractActionArgs.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.service.client.params; - -import com.beust.jcommander.Parameter; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; -import org.apache.hadoop.yarn.service.exceptions.ErrorStrings; -import org.apache.hadoop.yarn.service.exceptions.UsageException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; - -/** - * Base args for all actions - */ -public abstract class AbstractActionArgs extends ArgOps implements Arguments { - protected static final Logger log = - LoggerFactory.getLogger(AbstractActionArgs.class); - - - protected AbstractActionArgs() { - } - - /** - * URI/binding to the filesystem - */ - @Parameter(names = {ARG_FILESYSTEM, ARG_FILESYSTEM_LONG}, - description = "Filesystem Binding") - public String filesystemBinding; - - @Parameter(names = {ARG_BASE_PATH}, - description = "Service base path on the filesystem", - converter = PathArgumentConverter.class) - public Path basePath; - - /** - * This is the default parameter - */ - @Parameter - public final List parameters = new ArrayList<>(); - - /** - * get the name: relies on arg 1 being the cluster name in all operations - * @return the name argument, null if there is none - */ - public String getClusterName() { - return (parameters.isEmpty()) ? null : parameters.get(0); - } - - /** - -D name=value - - Define an configuration option which overrides any options in - the configuration XML files of the image or in the image configuration - directory. The values will be persisted. - Configuration options are only passed to the cluster when creating or reconfiguring a cluster. - - */ - - @Parameter(names = ARG_DEFINE, arity = 1, description = "Definitions") - public final List definitions = new ArrayList<>(); - - /** - * System properties - */ - @Parameter(names = {ARG_SYSPROP}, arity = 1, - description = "system properties in the form name value" + - " These are set after the JVM is started.") - public final List sysprops = new ArrayList<>(0); - - - @Parameter(names = {ARG_MANAGER_SHORT, ARG_MANAGER}, - description = "Binding (usually hostname:port) of the YARN resource manager") - public String manager; - - - @Parameter(names = ARG_DEBUG, description = "Debug mode") - public boolean debug = false; - - @Parameter(names = {ARG_HELP}, description = "Help", help = true) - public boolean help = false; - - /** - * Get the min #of params expected - * @return the min number of params in the {@link #parameters} field - */ - public int getMinParams() { - return 1; - } - - /** - * Get the name of the action - * @return the action name - */ - public abstract String getActionName() ; - - /** - * Get the max #of params expected - * @return the number of params in the {@link #parameters} field; - */ - public int getMaxParams() { - return getMinParams(); - } - - public void validate() throws BadCommandArgumentsException, UsageException { - - int minArgs = getMinParams(); - int actionArgSize = parameters.size(); - if (minArgs > actionArgSize) { - throw new BadCommandArgumentsException( - ErrorStrings.ERROR_NOT_ENOUGH_ARGUMENTS + getActionName() + - ", Expected minimum " + minArgs + " but got " + actionArgSize); - } - int maxArgs = getMaxParams(); - if (maxArgs == -1) { - maxArgs = minArgs; - } - if (actionArgSize > maxArgs) { - String message = String.format("%s for action %s: limit is %d but saw %d: ", - ErrorStrings.ERROR_TOO_MANY_ARGUMENTS, - getActionName(), maxArgs, - actionArgSize); - - log.error(message); - int index = 1; - StringBuilder buf = new StringBuilder(message); - for (String actionArg : parameters) { - log.error("[{}] \"{}\"", index++, actionArg); - buf.append(" \"").append(actionArg).append("\" "); - } - throw new BadCommandArgumentsException(buf.toString()); - } - } - - @Override - public String toString() { - return super.toString() + ": " + getActionName(); - } -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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/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 deleted file mode 100644 index 457e357..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/client/params/AbstractArgsDelegate.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.hadoop.yarn.service.client.params; - -import org.apache.hadoop.yarn.service.client.params.ArgOps; -import org.apache.hadoop.yarn.service.client.params.Arguments; - -/** - * Base class for all the delegates - */ -public class AbstractArgsDelegate extends ArgOps implements Arguments { -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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/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 deleted file mode 100644 index 017286f..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/client/params/AbstractClusterBuildingActionArgs.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.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.exceptions.BadCommandArgumentsException; - -import java.io.File; -import java.util.List; -import java.util.Map; - -/** - * Abstract Action to build things; shares args across build and - * list - */ -public abstract class AbstractClusterBuildingActionArgs - extends AbstractActionArgs { - @Parameter(names = {ARG_APPDEF}, - description = "Template application definition file in JSON format.") - public File appDef; - - public File getAppDef() { - return appDef; - } - - @Parameter(names = { - ARG_QUEUE }, description = "Queue to submit the application") - public String queue; - - @Parameter(names = { - ARG_LIFETIME }, description = "Lifetime of the application from the time of request") - public long lifetime; - - @ParametersDelegate - public ComponentArgsDelegate componentDelegate = new ComponentArgsDelegate(); - - @ParametersDelegate - public OptionArgsDelegate optionsDelegate = - new OptionArgsDelegate(); -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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 deleted file mode 100644 index c2ff545..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/client/params/ActionBuildArgs.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.hadoop.yarn.service.client.params; - -import com.beust.jcommander.Parameters; - -@Parameters(commandNames = { SliderActions.ACTION_BUILD}, - commandDescription = SliderActions.DESCRIBE_ACTION_BUILD) - -public class ActionBuildArgs extends AbstractClusterBuildingActionArgs { - - @Override - public String getActionName() { - return SliderActions.ACTION_BUILD; - } -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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/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 deleted file mode 100644 index 0097b4e..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/client/params/ActionClientArgs.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.service.client.params; - -import com.beust.jcommander.Parameter; -import com.beust.jcommander.Parameters; -import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; -import org.apache.hadoop.yarn.service.client.params.SliderActions; - -import java.io.File; - -@Parameters(commandNames = { SliderActions.ACTION_CLIENT}, - commandDescription = SliderActions.DESCRIBE_ACTION_CLIENT) - -public class ActionClientArgs extends AbstractActionArgs { - - @Override - public String getActionName() { - return SliderActions.ACTION_CLIENT; - } - - @Parameter(names = {ARG_INSTALL}, - description = "Install client") - public boolean install; - - @Parameter(names = {ARG_NAME}, - description = "The name of the application") - public String name; - - @Parameter(names = {ARG_PACKAGE}, - description = "Path to app package") - public String packageURI; - - @Parameter(names = {ARG_DEST}, - description = "The location where to install the client") - public File installLocation; - - @Parameter(names = {ARG_CONFIG}, - description = "Client configuration") - public File clientConfig; - - /** - * Get the min #of params expected - * - * @return the min number of params in the {@link #parameters} field - */ - public int getMinParams() { - return 0; - } - - @Override - public int getMaxParams() { - return 1; - } -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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 deleted file mode 100644 index eecffb6..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/client/params/ActionCreateArgs.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.hadoop.yarn.service.client.params; - -import com.beust.jcommander.Parameters; - -@Parameters(commandNames = { SliderActions.ACTION_CREATE}, - commandDescription = SliderActions.DESCRIBE_ACTION_CREATE) - -public class ActionCreateArgs extends AbstractClusterBuildingActionArgs { - - @Override - public String getActionName() { - return SliderActions.ACTION_CREATE; - } -} - http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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 deleted file mode 100644 index 51e07c9..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/client/params/ActionDependencyArgs.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.yarn.service.client.params; - -import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; -import org.apache.hadoop.yarn.service.exceptions.UsageException; - -import com.beust.jcommander.Parameter; -import com.beust.jcommander.Parameters; - -@Parameters(commandNames = { SliderActions.ACTION_DEPENDENCY }, - commandDescription = SliderActions.DESCRIBE_ACTION_DEPENDENCY) -public class ActionDependencyArgs extends AbstractActionArgs { - - @Override - public String getActionName() { - return SliderActions.ACTION_DEPENDENCY; - } - - @Parameter(names = { ARG_UPLOAD }, - description = "Upload AM and agent libraries to HDFS for this client") - public boolean upload; - - @Parameter(names = { ARG_OVERWRITE }, - description = "Overwrite current uploaded dependency libs") - public boolean overwrite = false; - - /** - * Get the min #of params expected - * - * @return the min number of params in the {@link #parameters} field - */ - public int getMinParams() { - return 0; - } - - @Override - public int getMaxParams() { - return 1; - } - - @Override - public void validate() throws BadCommandArgumentsException, UsageException { - super.validate(); - - if (!upload) { - throw new UsageException("Option " + ARG_UPLOAD + " is mandatory"); - } - } -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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/ActionDestroyArgs.java ---------------------------------------------------------------------- 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/ActionDestroyArgs.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/ActionDestroyArgs.java deleted file mode 100644 index 8c41c04..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/client/params/ActionDestroyArgs.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.hadoop.yarn.service.client.params; - -import com.beust.jcommander.Parameter; -import com.beust.jcommander.Parameters; - -@Parameters(commandNames = { SliderActions.ACTION_DESTROY}, - commandDescription = SliderActions.DESCRIBE_ACTION_DESTROY) - -public class ActionDestroyArgs extends AbstractActionArgs { - - @Override - public String getActionName() { - return SliderActions.ACTION_DESTROY; - } - - @Parameter(names = {ARG_FORCE}, - description = "force the operation") - public boolean force; -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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/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 deleted file mode 100644 index ba3c5a9..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/client/params/ActionExistsArgs.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.hadoop.yarn.service.client.params; - -import com.beust.jcommander.Parameter; -import com.beust.jcommander.Parameters; -import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; -import org.apache.hadoop.yarn.service.client.params.SliderActions; - -import java.io.File; - -@Parameters(commandNames = { SliderActions.ACTION_EXISTS}, - commandDescription = SliderActions.DESCRIBE_ACTION_EXISTS) - -public class ActionExistsArgs extends AbstractActionArgs { - - @Override - public String getActionName() { - return SliderActions.ACTION_EXISTS; - } - - @Parameter(names = {ARG_LIVE}, - description = "verify that the application is running") - public boolean live; - - @Parameter(names = {ARG_STATE}, - description = "verify that the application is in the specific YARN state") - public String state = ""; - - @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT}, - description = "output file for any application report") - public File out; -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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 deleted file mode 100644 index b7acf58..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/client/params/ActionFlexArgs.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.hadoop.yarn.service.client.params; - -import com.beust.jcommander.Parameters; -import com.beust.jcommander.ParametersDelegate; -import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException; - -import java.util.List; -import java.util.Map; - -@Parameters(commandNames = { SliderActions.ACTION_FLEX}, - commandDescription = SliderActions.DESCRIBE_ACTION_FLEX) - -public class ActionFlexArgs extends AbstractActionArgs { - - @Override - public String getActionName() { - return SliderActions.ACTION_FLEX; - } - - @ParametersDelegate - public ComponentArgsDelegate componentDelegate = new ComponentArgsDelegate(); - - /** - * Get the component mapping (may be empty, but never null) - * @return mapping - * @throws BadCommandArgumentsException parse problem - */ - public Map getComponentMap() throws - BadCommandArgumentsException { - return componentDelegate.getComponentMap(); - } -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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/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 deleted file mode 100644 index aecf0eb..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/client/params/ActionFreezeArgs.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.hadoop.yarn.service.client.params; - -import com.beust.jcommander.Parameter; -import com.beust.jcommander.Parameters; -import com.beust.jcommander.ParametersDelegate; - -@Parameters(commandNames = { SliderActions.ACTION_STOP }, - commandDescription = SliderActions.DESCRIBE_ACTION_FREEZE) - -public class ActionFreezeArgs extends AbstractActionArgs implements - WaitTimeAccessor { - @Override - public String getActionName() { - return SliderActions.ACTION_STOP; - } - - public static final String FREEZE_COMMAND_ISSUED = "stop command issued"; - @ParametersDelegate - public WaitArgsDelegate waitDelegate = new WaitArgsDelegate(); - - @Override - public int getWaittime() { - return waitDelegate.getWaittime(); - } - - @Override - public void setWaittime(int waittime) { - waitDelegate.setWaittime(waittime); - } - - @Parameter(names={ARG_MESSAGE}, - description = "reason for the operation") - public String message = FREEZE_COMMAND_ISSUED; - - @Parameter(names = {ARG_FORCE}, - description = "force the operation") - public boolean force; -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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/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 deleted file mode 100644 index 51aa88a..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/client/params/ActionHelpArgs.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.hadoop.yarn.service.client.params; - -import com.beust.jcommander.Parameters; -import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; -import org.apache.hadoop.yarn.service.client.params.SliderActions; - -/** - * The Help command - */ -@Parameters(commandNames = { SliderActions.ACTION_HELP}, - commandDescription = SliderActions.DESCRIBE_ACTION_HELP) -public class ActionHelpArgs extends AbstractActionArgs { - @Override - public String getActionName() { - return SliderActions.ACTION_HELP; - } - - /** - * Get the min #of params expected - * @return the min number of params in the {@link #parameters} field - */ - @Override - public int getMinParams() { - return 0; - } -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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/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 deleted file mode 100644 index 061121e..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/client/params/ActionKDiagArgs.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.hadoop.yarn.service.client.params; - -import com.beust.jcommander.Parameter; -import com.beust.jcommander.Parameters; -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; -import java.util.List; - -@Parameters(commandNames = { SliderActions.ACTION_KDIAG}, - commandDescription = SliderActions.DESCRIBE_ACTION_KDIAG) - -public class ActionKDiagArgs extends AbstractActionArgs { - - @Override - public String getActionName() { - return SliderActions.ACTION_KDIAG; - } - - @Parameter(names = {ARG_SERVICES}, variableArity = true, - description =" list of services to check") - public List services = new ArrayList<>(); - - @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT}, - description = "output file for report") - public File out; - - @Parameter(names = {ARG_KEYTAB}, description = "keytab to use") - public File keytab; - - @Parameter(names = {ARG_KEYLEN}, description = "minimum key length") - public int keylen = 256; - - @Parameter(names = {ARG_PRINCIPAL}, description = "principal to log in from a keytab") - public String principal; - - @Parameter(names = {ARG_SECURE}, description = "Is security required") - public boolean secure = false; - - @Override - public int getMinParams() { - return 0; - } - - @Override - public void validate() throws BadCommandArgumentsException, UsageException { - super.validate(); - if (keytab != null && SliderUtils.isUnset(principal)) { - throw new UsageException("Missing argument " + ARG_PRINCIPAL); - } - if (keytab == null && SliderUtils.isSet(principal)) { - throw new UsageException("Missing argument " + ARG_KEYTAB); - } - } -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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/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 deleted file mode 100644 index 7e51457..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/client/params/ActionKeytabArgs.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.hadoop.yarn.service.client.params; - -import com.beust.jcommander.Parameter; -import com.beust.jcommander.Parameters; -import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; -import org.apache.hadoop.yarn.service.client.params.SliderActions; - -@Parameters(commandNames = { SliderActions.ACTION_KEYTAB}, - commandDescription = SliderActions.DESCRIBE_ACTION_KEYTAB) - -public class ActionKeytabArgs extends AbstractActionArgs { - - public ActionKeytabArgs() { - super(); - } - - @Override - public String getActionName() { - return SliderActions.ACTION_INSTALL_KEYTAB; - } - - @Parameter(names = {ARG_KEYTABINSTALL}, - description = "Install the keytab") - public boolean install; - - @Parameter(names = {ARG_KEYTABDELETE}, - description = "Delete the keytab") - public boolean delete; - - @Parameter(names = {ARG_KEYTABLIST}, - description = "List of installed keytabs") - public boolean list; - - @Parameter(names = {ARG_KEYTAB}, - description = "Path or name of the keytab") - public String keytab; - - @Parameter(names = {ARG_FOLDER}, - description = "The name of the folder in which to store the keytab") - public String folder; - - @Parameter(names = {ARG_OVERWRITE}, description = "Overwrite existing keytab") - public boolean overwrite = false; - - /** - * Get the min #of params expected - * @return the min number of params in the {@link #parameters} field - */ - public int getMinParams() { - return 0; - } - - @Override - public int getMaxParams() { - return 3; - } - -} http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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 ---------------------------------------------------------------------- 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/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 deleted file mode 100644 index 005c172..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/client/params/ActionListArgs.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.hadoop.yarn.service.client.params; - -import java.util.HashSet; -import java.util.Set; - -import com.beust.jcommander.Parameter; -import com.beust.jcommander.Parameters; -import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs; -import org.apache.hadoop.yarn.service.client.params.SliderActions; - -@Parameters(commandNames = { SliderActions.ACTION_LIST}, - commandDescription = SliderActions.DESCRIBE_ACTION_LIST) - -public class ActionListArgs extends AbstractActionArgs { - @Override - public String getActionName() { - return SliderActions.ACTION_LIST; - } - - @Parameter(names = {ARG_LIVE}, - description = "List only live application instances") - public boolean live; - - @Parameter(names = {ARG_STATE}, - description = "list only applications in the specific YARN state") - public String state = ""; - - @Parameter(names = {ARG_VERBOSE}, - description = "print out information in details") - public boolean verbose = false; - - @Parameter(names = {ARG_CONTAINERS}, - description = "List containers of an application instance") - public boolean containers; - - @Parameter(names = {ARG_VERSION}, - description = "Filter containers by app version (used with " + - ARG_CONTAINERS + ")") - public String version; - - @Parameter(names = {ARG_COMPONENTS}, variableArity = true, - description = "Filter containers by component names (used with " + - ARG_CONTAINERS + ")") - public Set components = new HashSet<>(0); - - /** - * Get the min #of params expected - * @return the min number of params in the {@link #parameters} field - */ - public int getMinParams() { - return 0; - } - - @Override - public int getMaxParams() { - return 1; - } -} --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org For additional commands, e-mail: common-commits-help@hadoop.apache.org