You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2016/12/07 21:10:24 UTC

[24/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/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
----------------------------------------------------------------------
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/slider/core/launch/AbstractLauncher.java
new file mode 100644
index 0000000..5a3eb3d
--- /dev/null
+++ b/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
@@ -0,0 +1,528 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 com.google.common.base.Preconditions;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.api.RoleKeys;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.tools.CoreFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.MapOperations;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.InvocationTargetException;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * Launcher of applications: base class
+ */
+public abstract class AbstractLauncher extends Configured {
+  private static final Logger log =
+    LoggerFactory.getLogger(AbstractLauncher.class);
+  public static final String CLASSPATH = "CLASSPATH";
+  /**
+   * Filesystem to use for the launch
+   */
+  protected final CoreFileSystem coreFileSystem;
+  /**
+   * Env vars; set up at final launch stage
+   */
+  protected final Map<String, String> envVars = new HashMap<>();
+  protected final MapOperations env = new MapOperations("env", envVars);
+  protected final ContainerLaunchContext containerLaunchContext =
+    Records.newRecord(ContainerLaunchContext.class);
+  protected final List<String> commands = new ArrayList<>(20);
+  protected final Map<String, LocalResource> localResources = new HashMap<>();
+  protected final Map<String, String> mountPaths = new HashMap<>();
+  private final Map<String, ByteBuffer> serviceData = new HashMap<>();
+  // security
+  protected final Credentials credentials;
+  protected LogAggregationContext logAggregationContext;
+  protected boolean yarnDockerMode = false;
+  protected String dockerImage;
+  protected String yarnContainerMountPoints;
+  protected String runPrivilegedContainer;
+
+
+  /**
+   * Create instance.
+   * @param conf configuration
+   * @param coreFileSystem filesystem
+   * @param credentials initial set of credentials -null is permitted
+   */
+  protected AbstractLauncher(Configuration conf,
+      CoreFileSystem coreFileSystem,
+      Credentials credentials) {
+    super(conf);
+    this.coreFileSystem = coreFileSystem;
+    this.credentials = credentials != null ? credentials: new Credentials();
+  }
+
+  /**
+   * Get the container. Until "completed", this isn't valid to launch.
+   * @return the container to launch
+   */
+  public ContainerLaunchContext getContainerLaunchContext() {
+    return containerLaunchContext;
+  }
+  
+  public void setYarnDockerMode(boolean yarnDockerMode){
+    this.yarnDockerMode = yarnDockerMode;
+  }
+
+  /**
+   * Get the env vars to work on
+   * @return env vars
+   */
+  public MapOperations getEnv() {
+    return env;
+  }
+
+  /**
+   * Get the launch commands.
+   * @return the live list of commands 
+   */
+  public List<String> getCommands() {
+    return commands;
+  }
+
+  /**
+   * Get the map of local resources.
+   * @return the live map of local resources.
+   */
+  public Map<String, LocalResource> getLocalResources() {
+    return localResources;
+  }
+
+  public void addLocalResource(String subPath, LocalResource resource) {
+    localResources.put(subPath, resource);
+  }
+
+  public void addLocalResource(String subPath, LocalResource resource, String mountPath) {
+    localResources.put(subPath, resource);
+    mountPaths.put(subPath, mountPath);
+  }
+
+  /**
+   * Add a set of local resources
+   * @param resourceMap map of name:resource to add
+   */
+  public void addLocalResources(Map<String, LocalResource> resourceMap) {
+    localResources.putAll(resourceMap);
+  }
+
+  public Map<String, ByteBuffer> getServiceData() {
+    return serviceData;
+  }
+
+  /**
+   * Accessor to the credentials
+   * @return the credentials associated with this launcher
+   */
+  public Credentials getCredentials() {
+    return credentials;
+  }
+
+  /**
+   * Add a command line. It is converted to a single command before being
+   * added.
+   * @param cmd
+   */
+  public void addCommandLine(CommandLineBuilder cmd) {
+    commands.add(cmd.build());
+  }
+
+  public void addCommand(String cmd) {
+    commands.add(cmd);
+  }
+
+  /**
+   * Add a list of commands. Each element in the list becomes a single command
+   * @param commandList list of commands
+   */
+  public void addCommands(List<String> commandList) {
+    commands.addAll(commandList);
+  }
+
+  /**
+   * Get all commands as a string, separated by ";". This is for diagnostics
+   * @return a string description of the commands
+   */
+  public String getCommandsAsString() {
+    return SliderUtils.join(getCommands(), "; ");
+  }
+
+  /**
+   * Complete the launch context (copy in env vars, etc).
+   * @return the container to launch
+   */
+  public ContainerLaunchContext completeContainerLaunch() throws IOException {
+    
+    String cmdStr = SliderUtils.join(commands, " ", false);
+    log.debug("Completed setting up container command {}", cmdStr);
+    containerLaunchContext.setCommands(commands);
+
+    //env variables
+    if (log.isDebugEnabled()) {
+      log.debug("Environment variables");
+      for (Map.Entry<String, String> envPair : envVars.entrySet()) {
+        log.debug("    \"{}\"=\"{}\"", envPair.getKey(), envPair.getValue());
+      }
+    }    
+    containerLaunchContext.setEnvironment(env);
+
+    //service data
+    if (log.isDebugEnabled()) {
+      log.debug("Service Data size");
+      for (Map.Entry<String, ByteBuffer> entry : serviceData.entrySet()) {
+        log.debug("\"{}\"=> {} bytes of data", entry.getKey(),
+            entry.getValue().array().length);
+      }
+    }
+    containerLaunchContext.setServiceData(serviceData);
+
+    // resources
+    dumpLocalResources();
+    containerLaunchContext.setLocalResources(localResources);
+
+    //tokens
+    log.debug("{} tokens", credentials.numberOfTokens());
+    containerLaunchContext.setTokens(CredentialUtils.marshallCredentials(
+        credentials));
+
+    if(yarnDockerMode){
+      Map<String, String> env = containerLaunchContext.getEnvironment();
+      env.put("YARN_CONTAINER_RUNTIME_TYPE", "docker");
+      env.put("YARN_CONTAINER_RUNTIME_DOCKER_IMAGE", dockerImage);//if yarnDockerMode, then dockerImage is set
+      env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER", runPrivilegedContainer);
+      StringBuilder sb = new StringBuilder();
+      for (Entry<String,String> mount : mountPaths.entrySet()) {
+        if (sb.length() > 0) {
+          sb.append(",");
+        }
+        sb.append(mount.getKey());
+        sb.append(":");
+        sb.append(mount.getValue());
+      }
+      env.put("YARN_CONTAINER_RUNTIME_DOCKER_LOCAL_RESOURCE_MOUNTS", sb.toString());
+      log.info("yarn docker env var has been set {}", containerLaunchContext.getEnvironment().toString());
+    }
+
+    return containerLaunchContext;
+  }
+
+  /**
+   * Dump local resources at debug level
+   */
+  private void dumpLocalResources() {
+    if (log.isDebugEnabled()) {
+      log.debug("{} resources: ", localResources.size());
+      for (Map.Entry<String, LocalResource> entry : localResources.entrySet()) {
+
+        String key = entry.getKey();
+        LocalResource val = entry.getValue();
+        log.debug(key + "=" + SliderUtils.stringify(val.getResource()));
+      }
+    }
+  }
+
+  /**
+   * This is critical for an insecure cluster -it passes
+   * down the username to YARN, and so gives the code running
+   * in containers the rights it needs to work with
+   * data.
+   * @throws IOException problems working with current user
+   */
+  protected void propagateUsernameInInsecureCluster() throws IOException {
+    //insecure cluster: propagate user name via env variable
+    String userName = UserGroupInformation.getCurrentUser().getUserName();
+    env.put(SliderKeys.HADOOP_USER_NAME, userName);
+  }
+
+  /**
+   * Extract any resource requirements from this component's settings.
+   * All fields that are set will override the existing values -if
+   * unset that resource field will be left unchanged.
+   *
+   * Important: the configuration must already be fully resolved 
+   * in order to pick up global options.
+   * @param resource resource to configure
+   * @param map map of options
+   */
+  public void extractResourceRequirements(Resource resource,
+                                          Map<String, String> map) {
+
+    if (map != null) {
+      MapOperations options = new MapOperations("", map);
+      resource.setMemory(options.getOptionInt(ResourceKeys.YARN_MEMORY,
+                                              resource.getMemory()));
+      resource.setVirtualCores(options.getOptionInt(ResourceKeys.YARN_CORES,
+                                                    resource.getVirtualCores()));
+    }
+  }
+
+  /**
+   * Extract the value for option
+   * {@code yarn.resourcemanager.am.retry-count-window-ms}
+   * and set it on the ApplicationSubmissionContext. Use the default value
+   * if option is not set.
+   *
+   * @param submissionContext
+   * @param map
+   */
+  public void extractAmRetryCount(ApplicationSubmissionContext submissionContext,
+                                  Map<String, String> map) {
+
+    if (map != null) {
+      MapOperations options = new MapOperations("", map);
+      long amRetryCountWindow = options.getOptionLong(ResourceKeys
+          .YARN_RESOURCEMANAGER_AM_RETRY_COUNT_WINDOW_MS,
+          ResourceKeys.DEFAULT_AM_RETRY_COUNT_WINDOW_MS);
+      log.info("Setting {} to {}",
+          ResourceKeys.YARN_RESOURCEMANAGER_AM_RETRY_COUNT_WINDOW_MS,
+          amRetryCountWindow);
+      submissionContext.setAttemptFailuresValidityInterval(amRetryCountWindow);
+    }
+  }
+
+  public void extractLogAggregationContext(Map<String, String> map) {
+    if (map != null) {
+      String logPatternSepStr = "\\|";
+      String logPatternJoinStr = "|";
+      MapOperations options = new MapOperations("", map);
+
+      List<String> logIncludePatterns = new ArrayList<>();
+      String includePatternExpression = options.getOption(
+          ResourceKeys.YARN_LOG_INCLUDE_PATTERNS, "").trim();
+      if (!includePatternExpression.isEmpty()) {
+        String[] includePatterns = includePatternExpression
+            .split(logPatternSepStr);
+        for (String includePattern : includePatterns) {
+          String trimmedIncludePattern = includePattern.trim();
+          if (!trimmedIncludePattern.isEmpty()) {
+            logIncludePatterns.add(trimmedIncludePattern);
+          }
+        }
+      }
+      String logIncludePattern = StringUtils.join(logIncludePatterns,
+          logPatternJoinStr);
+      log.info("Log include patterns: {}", logIncludePattern);
+
+      List<String> logExcludePatterns = new ArrayList<>();
+      String excludePatternExpression = options.getOption(
+          ResourceKeys.YARN_LOG_EXCLUDE_PATTERNS, "").trim();
+      if (!excludePatternExpression.isEmpty()) {
+        String[] excludePatterns = excludePatternExpression
+            .split(logPatternSepStr);
+        for (String excludePattern : excludePatterns) {
+          String trimmedExcludePattern = excludePattern.trim();
+          if (!trimmedExcludePattern.isEmpty()) {
+            logExcludePatterns.add(trimmedExcludePattern);
+          }
+        }
+      }
+      String logExcludePattern = StringUtils.join(logExcludePatterns,
+          logPatternJoinStr);
+      log.info("Log exclude patterns: {}", logExcludePattern);
+
+      // SLIDER-810/YARN-3154 - hadoop 2.7.0 onwards a new instance method has
+      // been added for log aggregation for LRS. Existing newInstance method's
+      // behavior has changed and is used for log aggregation only after the
+      // application has finished. This forces Slider users to move to hadoop
+      // 2.7.0+ just for log aggregation, which is not very desirable. So we
+      // decided to use reflection here to find out if the new 2.7.0 newInstance
+      // method is available. If yes, then we use it, so log aggregation will
+      // work in hadoop 2.7.0+ env. If no, then we fallback to the pre-2.7.0
+      // newInstance method, which means log aggregation will work as expected
+      // in hadoop 2.6 as well.
+      // TODO: At some point, say 2-3 Slider releases down, when most users are
+      // running hadoop 2.7.0, we should get rid of the reflection code here.
+      try {
+        Method logAggregationContextMethod = LogAggregationContext.class
+            .getMethod("newInstance", String.class, String.class, String.class,
+                String.class);
+        // Need to set include/exclude patterns appropriately since by default
+        // rolled log aggregation is not done for any files, so defaults are
+        // - include pattern set to ""
+        // - exclude pattern set to "*"
+        // For Slider we want all logs to be uploaded if include/exclude
+        // patterns are left empty by the app owner in resources file
+        if (StringUtils.isEmpty(logIncludePattern)
+            && StringUtils.isEmpty(logExcludePattern)) {
+          logIncludePattern = ".*";
+          logExcludePattern = "";
+        } else if (StringUtils.isEmpty(logIncludePattern)
+            && StringUtils.isNotEmpty(logExcludePattern)) {
+          logIncludePattern = ".*";
+        } else if (StringUtils.isNotEmpty(logIncludePattern)
+            && StringUtils.isEmpty(logExcludePattern)) {
+          logExcludePattern = "";
+        }
+        log.debug("LogAggregationContext newInstance method for rolled logs "
+            + "include/exclude patterns is available");
+        log.info("Modified log include patterns: {}", logIncludePattern);
+        log.info("Modified log exclude patterns: {}", logExcludePattern);
+        logAggregationContext = (LogAggregationContext) logAggregationContextMethod
+            .invoke(null, null, null, logIncludePattern, logExcludePattern);
+      } catch (NoSuchMethodException | SecurityException
+          | IllegalAccessException | IllegalArgumentException
+          | InvocationTargetException e) {
+        log.debug("LogAggregationContext newInstance method for rolled logs "
+            + "include/exclude patterns is not available - fallback to old one");
+        log.debug(e.toString());
+        logAggregationContext = LogAggregationContext.newInstance(
+            logIncludePattern, logExcludePattern);
+      }
+    }
+  }
+
+  /**
+   * Utility method to set up the classpath
+   * @param classpath classpath to use
+   */
+  public void setClasspath(ClasspathConstructor classpath) {
+    setEnv(CLASSPATH, classpath.buildClasspath());
+  }
+
+  /**
+   * Set an environment variable in the launch context
+   * @param var variable name
+   * @param value value (must be non null)
+   */
+  public void setEnv(String var, String value) {
+    Preconditions.checkArgument(var != null, "null variable name");
+    Preconditions.checkArgument(value != null, "null value");
+    env.put(var, value);
+  }
+
+  /**
+   * Set an environment variable if its value is non-null.
+   * @param var variable name
+   * @param value value (may be null)
+   */
+  public void maybeSetEnv(String var, String value) {
+    if (value != null) {
+      setEnv(var, value);
+    }
+  }
+
+  public void putEnv(Map<String, String> map) {
+    env.putAll(map);
+  }
+
+  /**
+   * Important: the configuration must already be fully resolved 
+   * in order to pick up global options
+   * Copy env vars into the launch context.
+   */
+  public boolean copyEnvVars(MapOperations options) {
+    if (options == null) {
+      return false;
+    }
+    for (Map.Entry<String, String> entry : options.entrySet()) {
+      String key = entry.getKey();
+      if (key.startsWith(RoleKeys.ENV_PREFIX)) {
+        key = key.substring(RoleKeys.ENV_PREFIX.length());
+        env.put(key, entry.getValue());
+      }
+    }
+    return true;
+  }
+
+  public String[] dumpEnvToString() {
+
+    List<String> nodeEnv = new ArrayList<>();
+
+    for (Map.Entry<String, String> entry : env.entrySet()) {
+      String envElt = String.format("%s=\"%s\"",
+                                    entry.getKey(),
+                                    entry.getValue());
+      log.debug(envElt);
+      nodeEnv.add(envElt);
+    }
+    String[] envDescription = nodeEnv.toArray(new String[nodeEnv.size()]);
+
+    return envDescription;
+  }
+
+  /**
+   * Submit an entire directory
+   * @param srcDir src path in filesystem
+   * @param destRelativeDir relative path under destination local dir
+   * @throws IOException IO problems
+   */
+  public void submitDirectory(Path srcDir, String destRelativeDir)
+      throws IOException {
+    //add the configuration resources
+    Map<String, LocalResource> confResources;
+    confResources = coreFileSystem.submitDirectory(
+      srcDir,
+      destRelativeDir);
+    addLocalResources(confResources);
+  }
+
+  /**
+   * Return the label expression and if not set null
+   * @param map map to look up
+   * @return extracted label or null
+   */
+  public String extractLabelExpression(Map<String, String> map) {
+    if (map != null) {
+      MapOperations options = new MapOperations("", map);
+      return options.getOption(ResourceKeys.YARN_LABEL_EXPRESSION, null);
+    }
+    return null;
+  }
+
+  public void setDockerImage(String dockerImage) {
+    this.dockerImage = dockerImage;
+  }
+
+  public void setYarnContainerMountPoints(String yarnContainerMountPoints) {
+    this.yarnContainerMountPoints = yarnContainerMountPoints;
+  }
+
+  public void setRunPrivilegedContainer(String runPrivilegedContainer) {
+    this.runPrivilegedContainer = runPrivilegedContainer;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AppMasterLauncher.java
----------------------------------------------------------------------
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/AppMasterLauncher.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/AppMasterLauncher.java
new file mode 100644
index 0000000..7190c3a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AppMasterLauncher.java
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+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.util.Records;
+import org.apache.slider.client.SliderYarnClientImpl;
+import org.apache.slider.common.tools.CoreFileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
+public class AppMasterLauncher extends AbstractLauncher {
+
+
+  private static final Logger log =
+    LoggerFactory.getLogger(AppMasterLauncher.class);
+
+  public final YarnClientApplication application;
+  public final String name;
+  public final String type;
+  public final ApplicationSubmissionContext submissionContext;
+  public final ApplicationId appId;
+  public final boolean secureCluster;
+  private int maxAppAttempts = 0;
+  private boolean keepContainersOverRestarts = true;
+  private String queue = YarnConfiguration.DEFAULT_QUEUE_NAME;
+  private int priority = 1;
+  private final Resource resource = Records.newRecord(Resource.class);
+  private final SliderYarnClientImpl yarnClient;
+  private Long submitTime;
+
+  /**
+   * Build the AM Launcher
+   * @param name app name
+   * @param type application type
+   * @param conf hadoop config
+   * @param fs filesystem binding
+   * @param yarnClient yarn client
+   * @param secureCluster flag to indicate secure cluster
+   * @param options map of options. All values are extracted in this constructor only
+   * @param resourceGlobalOptions global options
+   * @param applicationTags any app tags
+   * @param credentials initial set of credentials
+   * @throws IOException
+   * @throws YarnException
+   */
+  public AppMasterLauncher(String name,
+      String type,
+      Configuration conf,
+      CoreFileSystem fs,
+      SliderYarnClientImpl yarnClient,
+      boolean secureCluster,
+      Map<String, String> options,
+      Map<String, String> resourceGlobalOptions,
+      Set<String> applicationTags,
+      Credentials credentials) throws IOException, YarnException {
+    super(conf, fs, credentials);
+    this.yarnClient = yarnClient;
+    this.application = yarnClient.createApplication();
+    this.name = name;
+    this.type = type;
+    this.secureCluster = secureCluster;
+
+    submissionContext = application.getApplicationSubmissionContext();
+    appId = submissionContext.getApplicationId();
+    // set the application name;
+    submissionContext.setApplicationName(name);
+    // app type used in service enum;
+    submissionContext.setApplicationType(type);
+    if (!applicationTags.isEmpty()) {
+      submissionContext.setApplicationTags(applicationTags);
+    }
+    submissionContext.setNodeLabelExpression(extractLabelExpression(options));
+
+    extractAmRetryCount(submissionContext, resourceGlobalOptions);
+    extractResourceRequirements(resource, options);
+    extractLogAggregationContext(resourceGlobalOptions);
+  }
+
+  public void setMaxAppAttempts(int maxAppAttempts) {
+    this.maxAppAttempts = maxAppAttempts;
+  }
+
+  public void setKeepContainersOverRestarts(boolean keepContainersOverRestarts) {
+    this.keepContainersOverRestarts = keepContainersOverRestarts;
+  }
+
+
+  public Resource getResource() {
+    return resource;
+  }
+
+  public void setMemory(int memory) {
+    resource.setMemory(memory);
+  }
+
+  public void setVirtualCores(int cores) {
+    resource.setVirtualCores(cores);
+  }
+
+  public ApplicationId getApplicationId() {
+    return appId;
+  }
+
+  public int getMaxAppAttempts() {
+    return maxAppAttempts;
+  }
+
+  public boolean isKeepContainersOverRestarts() {
+    return keepContainersOverRestarts;
+  }
+
+  public String getQueue() {
+    return queue;
+  }
+
+  public int getPriority() {
+    return priority;
+  }
+
+  public void setQueue(String queue) {
+    this.queue = queue;
+  }
+
+  public void setPriority(int priority) {
+    this.priority = priority;
+  }
+
+  /**
+   * Complete the launch context (copy in env vars, etc).
+   * @return the container to launch
+   */
+  public ApplicationSubmissionContext completeAppMasterLaunch()
+      throws IOException {
+
+    //queue priority
+    Priority pri = Records.newRecord(Priority.class);
+    pri.setPriority(priority);
+    submissionContext.setPriority(pri);
+
+    // Set the queue to which this application is to be submitted in the RM
+    // Queue for App master
+
+    submissionContext.setQueue(queue);
+
+
+    //container requirements
+    submissionContext.setResource(resource);
+    submissionContext.setLogAggregationContext(logAggregationContext);
+
+    if (keepContainersOverRestarts) {
+      log.debug("Requesting cluster stays running over AM failure");
+      submissionContext.setKeepContainersAcrossApplicationAttempts(true);
+    }
+
+    if (maxAppAttempts > 0) {
+      log.debug("Setting max AM attempts to {}", maxAppAttempts);
+      submissionContext.setMaxAppAttempts(maxAppAttempts);
+    }
+
+    if (secureCluster) {
+      //tokens
+      log.debug("Credentials: {}",
+          CredentialUtils.dumpTokens(getCredentials(), "\n"));
+
+    } else {
+      propagateUsernameInInsecureCluster();
+    }
+    completeContainerLaunch();
+    submissionContext.setAMContainerSpec(containerLaunchContext);
+    return submissionContext;
+  }
+
+  /**
+   * Submit the application. 
+   * @return a launched application representing the submitted application
+   * @throws IOException
+   * @throws YarnException
+   */
+  public LaunchedApplication submitApplication() throws IOException, YarnException {
+    completeAppMasterLaunch();
+    log.info("Submitting application to Resource Manager");
+    ApplicationId applicationId =
+      yarnClient.submitApplication(submissionContext);
+    // implicit success; record the time
+    submitTime = System.currentTimeMillis();
+    return new LaunchedApplication(applicationId, yarnClient);
+  }
+
+  /**
+   * Build a serializable application report. This is a very minimal
+   * report that contains the application Id, name and type \u2014the information
+   * available
+   * @return a data structure which can be persisted
+   */
+  public SerializedApplicationReport createSerializedApplicationReport() {
+    SerializedApplicationReport sar = new SerializedApplicationReport();
+    sar.applicationId = appId.toString();
+    sar.name = name;
+    sar.applicationType = type;
+    sar.queue = queue;
+    sar.submitTime = submitTime;
+    return sar;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/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
----------------------------------------------------------------------
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/slider/core/launch/ClasspathConstructor.java
new file mode 100644
index 0000000..6eb4058
--- /dev/null
+++ b/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
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.StringUtils;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.slider.common.tools.SliderUtils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * build a classpath -allows for entries to be injected in front of
+ * YARN classpath as well as behind, adds appropriate separators, 
+ * extraction of local classpath, etc.
+ */
+public class ClasspathConstructor {
+
+    public static final String CLASS_PATH_SEPARATOR = ApplicationConstants.CLASS_PATH_SEPARATOR;
+  private final List<String> pathElements = new ArrayList<>();
+
+  public ClasspathConstructor() {
+  }
+
+
+  /**
+   * Get the list of JARs from the YARN settings
+   * @param config configuration
+   */
+  public List<String> yarnApplicationClasspath(Configuration config) {
+    String[] cp = config.getTrimmedStrings(
+      YarnConfiguration.YARN_APPLICATION_CLASSPATH,
+      YarnConfiguration.DEFAULT_YARN_CROSS_PLATFORM_APPLICATION_CLASSPATH);
+    return cp != null ? Arrays.asList(cp) : new ArrayList<String>(0);
+
+  }
+
+
+  @Override
+  public String toString() {
+    return buildClasspath();
+  }
+
+  public String buildClasspath() {
+    return SliderUtils.join(pathElements,
+        CLASS_PATH_SEPARATOR,
+        false);
+  }
+
+  /**
+   * Get a copy of the path list
+   * @return the JARs
+   */
+  public List<String> getPathElements() {
+    return Collections.unmodifiableList(pathElements);
+  }
+
+  /**
+   * Append an entry
+   * @param path path
+   */
+  public void append(String path) {
+    pathElements.add(path);
+  }
+
+  /**
+   * Insert a path at the front of the list. This places it ahead of
+   * the standard YARN artifacts
+   * @param path path to the JAR. Absolute or relative -on the target
+   * system
+   */
+  public void insert(String path) {
+    pathElements.add(0, path);
+  }
+
+  public void appendAll(Collection<String> paths) {
+    pathElements.addAll(paths);
+  }
+
+  public void insertAll(Collection<String> paths) {
+    pathElements.addAll(0, paths);
+  }
+
+
+  public void addLibDir(String pathToLibDir) {
+    append(buildLibDir(pathToLibDir));
+  }
+
+  public void insertLibDir(String pathToLibDir) {
+    insert(buildLibDir(pathToLibDir));
+  }
+
+  public void addClassDirectory(String pathToDir) {
+    append(appendDirectoryTerminator(pathToDir));
+  }
+
+  public void insertClassDirectory(String pathToDir) {
+    insert(buildLibDir(appendDirectoryTerminator(pathToDir)));
+  }
+
+
+  public void addRemoteClasspathEnvVar() {
+    append(ApplicationConstants.Environment.CLASSPATH.$$());
+  }
+
+
+  public void insertRemoteClasspathEnvVar() {
+    append(ApplicationConstants.Environment.CLASSPATH.$$());
+  }
+
+
+  /**
+   * Build a lib dir path
+   * @param pathToLibDir path to the directory; may or may not end with a
+   * trailing space
+   * @return a path to a lib dir that is compatible with the java classpath
+   */
+  public String buildLibDir(String pathToLibDir) {
+    String dir = appendDirectoryTerminator(pathToLibDir);
+    dir += "*";
+    return dir;
+  }
+
+  private String appendDirectoryTerminator(String pathToLibDir) {
+    String dir = pathToLibDir.trim();
+    if (!dir.endsWith("/")) {
+      dir += "/";
+    }
+    return dir;
+  }
+
+  /**
+   * Split a classpath. This uses the local path separator so MUST NOT
+   * be used to work with remote classpaths
+   * @param localpath local path
+   * @return a splite
+   */
+  public Collection<String> splitClasspath(String localpath) {
+    String separator = System.getProperty("path.separator");
+    return StringUtils.getStringCollection(localpath, separator);
+  }
+
+  /**
+   * Get the local JVM classpath split up
+   * @return the list of entries on the JVM classpath env var
+   */
+  public Collection<String> localJVMClasspath() {
+    return splitClasspath(System.getProperty("java.class.path"));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/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
----------------------------------------------------------------------
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/slider/core/launch/CommandLineBuilder.java
new file mode 100644
index 0000000..dbaa981
--- /dev/null
+++ b/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
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.slider.common.tools.SliderUtils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Build a single command line to include in the container commands;
+ * Special support for JVM command buildup.
+ */
+public class CommandLineBuilder {
+  protected final List<String> argumentList = new ArrayList<>(20);
+
+
+  /**
+   * Add an entry to the command list
+   * @param args arguments -these will be converted strings
+   */
+  public void add(Object... args) {
+    for (Object arg : args) {
+      argumentList.add(arg.toString());
+    }
+  }
+
+  /**
+   * Get the value at an offset
+   * @param offset offset
+   * @return the value at that point
+   */
+  public String elt(int offset) {
+    return argumentList.get(offset);
+  }
+
+  /**
+   * Get the number of arguments
+   * @return an integer >= 0
+   */
+  public int size() {
+    return argumentList.size();
+  }
+  
+  /**
+   * Append the output and error files to the tail of the command
+   * @param stdout out
+   * @param stderr error. Set this to null to append into stdout
+   */
+  public void addOutAndErrFiles(String stdout, String stderr) {
+    Preconditions.checkNotNull(stdout, "Null output file");
+    Preconditions.checkState(!stdout.isEmpty(), "output filename invalid");
+    // write out the path output
+    argumentList.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/" +
+             stdout);
+    if (stderr != null) {
+      argumentList.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/" +
+               stderr);
+    } else {
+      argumentList.add("2>&1");
+    }
+  }
+
+  /**
+   * This just returns the command line
+   * @see #build()
+   * @return the command line
+   */
+  @Override
+  public String toString() {
+    return build();
+  }
+
+  /**
+   * Build the command line
+   * @return the command line
+   */
+  public String build() {
+    return SliderUtils.join(argumentList, " ");
+  }
+
+  public List<String> getArgumentList() {
+    return argumentList;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..d220928
--- /dev/null
+++ 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
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+  // Allocated container
+  public final Container container;
+
+  public ContainerLauncher(Configuration conf,
+      CoreFileSystem coreFileSystem,
+      Container container,
+      Credentials credentials) {
+    super(conf, coreFileSystem, credentials);
+    this.container = container;
+  }
+
+  /**
+   * This code is in the dist shell examples -it's been moved here
+   * so that if it is needed, it's still here
+   * @return a remote user with a token to access the container.
+   */
+  public UserGroupInformation setupUGI() {
+    UserGroupInformation user =
+      UserGroupInformation.createRemoteUser(container.getId().toString());
+    String cmIpPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort();
+    final InetSocketAddress cmAddress = NetUtils.createSocketAddr(cmIpPortStr);
+
+    org.apache.hadoop.yarn.api.records.Token containerToken = container.getContainerToken();
+    if (containerToken != null) {
+      Token<ContainerTokenIdentifier> token =
+        ConverterUtils.convertFromYarn(containerToken, cmAddress);
+      user.addToken(token);
+    }
+    return user;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/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
----------------------------------------------------------------------
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/slider/core/launch/CredentialUtils.java
new file mode 100644
index 0000000..5357cc4
--- /dev/null
+++ b/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
@@ -0,0 +1,379 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+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.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
+import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.HAUtil;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+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.slider.common.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;
+import java.nio.ByteBuffer;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+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.*;
+
+/**
+ * Utils to work with credentials and tokens.
+ *
+ * Designed to be movable to Hadoop core
+ */
+public final class CredentialUtils {
+
+  public static final String JOB_CREDENTIALS_BINARY
+      = SliderXmlConfKeys.MAPREDUCE_JOB_CREDENTIALS_BINARY;
+
+  private CredentialUtils() {
+  }
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CredentialUtils.class);
+
+  /**
+   * Save credentials to a byte buffer. Returns null if there were no
+   * credentials to save
+   * @param credentials credential set
+   * @return a byte buffer of serialized tokens
+   * @throws IOException if the credentials could not be written to the stream
+   */
+  public static ByteBuffer marshallCredentials(Credentials credentials) throws IOException {
+    ByteBuffer buffer = null;
+    if (!credentials.getAllTokens().isEmpty()) {
+      DataOutputBuffer dob = new DataOutputBuffer();
+      try {
+        credentials.writeTokenStorageToStream(dob);
+      } finally {
+        dob.close();
+      }
+      buffer = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+    }
+    return buffer;
+  }
+
+  public static File locateEnvCredentials(Map<String, String> 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<String, String> 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)
+   * @param credentials credentials to write
+   * @throws IOException
+   */
+  public static void saveTokens(File file,
+      Credentials credentials) throws IOException {
+    try(DataOutputStream daos = new DataOutputStream(
+        new FileOutputStream(file))) {
+      credentials.writeTokenStorageToStream(daos);
+    }
+  }
+
+  /**
+   * Look up and return the resource manager's principal. This method
+   * automatically does the <code>_HOST</code> replacement in the principal and
+   * correctly handles HA resource manager configurations.
+   *
+   * From: YARN-4629
+   * @param conf the {@link Configuration} file from which to read the
+   * principal
+   * @return the resource manager's principal string
+   * @throws IOException thrown if there's an error replacing the host name
+   */
+  public static String getRMPrincipal(Configuration conf) throws IOException {
+    String principal = conf.get(RM_PRINCIPAL, "");
+    String hostname;
+    Preconditions.checkState(!principal.isEmpty(), "Not set: " + RM_PRINCIPAL);
+
+    if (HAUtil.isHAEnabled(conf)) {
+      YarnConfiguration yarnConf = new YarnConfiguration(conf);
+      if (yarnConf.get(RM_HA_ID) == null) {
+        // If RM_HA_ID is not configured, use the first of RM_HA_IDS.
+        // Any valid RM HA ID should work.
+        String[] rmIds = yarnConf.getStrings(RM_HA_IDS);
+        Preconditions.checkState((rmIds != null) && (rmIds.length > 0),
+            "Not set " + RM_HA_IDS);
+        yarnConf.set(RM_HA_ID, rmIds[0]);
+      }
+
+      hostname = yarnConf.getSocketAddr(
+          RM_ADDRESS,
+          DEFAULT_RM_ADDRESS,
+          DEFAULT_RM_PORT).getHostName();
+    } else {
+      hostname = conf.getSocketAddr(
+          RM_ADDRESS,
+          DEFAULT_RM_ADDRESS,
+          DEFAULT_RM_PORT).getHostName();
+    }
+    return SecurityUtil.getServerPrincipal(principal, hostname);
+  }
+
+  /**
+   * Create and add any filesystem delegation tokens with
+   * the RM(s) configured to be able to renew them. Returns null
+   * on an insecure cluster (i.e. harmless)
+   * @param conf configuration
+   * @param fs filesystem
+   * @param credentials credentials to update
+   * @return a list of all added tokens.
+   * @throws IOException
+   */
+  public static Token<?>[] addRMRenewableFSDelegationTokens(Configuration conf,
+      FileSystem fs,
+      Credentials credentials) throws IOException {
+    Preconditions.checkArgument(conf != null);
+    Preconditions.checkArgument(credentials != null);
+    if (UserGroupInformation.isSecurityEnabled()) {
+      return fs.addDelegationTokens(CredentialUtils.getRMPrincipal(conf),
+          credentials);
+    }
+    return null;
+  }
+
+  /**
+   * Add an FS delegation token which can be renewed by the current user
+   * @param fs filesystem
+   * @param credentials credentials to update
+   * @throws IOException problems.
+   */
+  public static void addSelfRenewableFSDelegationTokens(
+      FileSystem fs,
+      Credentials credentials) throws IOException {
+    Preconditions.checkArgument(fs != null);
+    Preconditions.checkArgument(credentials != null);
+    fs.addDelegationTokens(
+        getSelfRenewer(),
+        credentials);
+  }
+
+  public static String getSelfRenewer() throws IOException {
+    return UserGroupInformation.getLoginUser().getShortUserName();
+  }
+
+  /**
+   * Create and add an RM delegation token to the credentials
+   * @param yarnClient Yarn Client
+   * @param credentials to add token to
+   * @return the token which was added
+   * @throws IOException
+   * @throws YarnException
+   */
+  public static Token<TokenIdentifier> addRMDelegationToken(YarnClient yarnClient,
+      Credentials credentials)
+      throws IOException, YarnException {
+    Configuration conf = yarnClient.getConfig();
+    Text rmPrincipal = new Text(CredentialUtils.getRMPrincipal(conf));
+    Text rmDTService = ClientRMProxy.getRMDelegationTokenService(conf);
+    Token<TokenIdentifier> rmDelegationToken =
+        ConverterUtils.convertFromYarn(
+            yarnClient.getRMDelegationToken(rmPrincipal),
+            rmDTService);
+    credentials.addToken(rmDelegationToken.getService(), rmDelegationToken);
+    return rmDelegationToken;
+  }
+
+  public static Token<TimelineDelegationTokenIdentifier> maybeAddTimelineToken(
+      Configuration conf,
+      Credentials credentials)
+      throws IOException, YarnException {
+    if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false)) {
+      LOG.debug("Timeline service enabled -fetching token");
+
+      try(TimelineClient timelineClient = TimelineClient.createTimelineClient()) {
+        timelineClient.init(conf);
+        timelineClient.start();
+        Token<TimelineDelegationTokenIdentifier> token =
+            timelineClient.getDelegationToken(
+                CredentialUtils.getRMPrincipal(conf));
+        credentials.addToken(token.getService(), token);
+        return token;
+      }
+    } else {
+      LOG.debug("Timeline service is disabled");
+      return null;
+    }
+  }
+
+  /**
+   * Filter a list of tokens from a set of credentials
+   * @param credentials credential source (a new credential set os re
+   * @param filter List of tokens to strip out
+   * @return a new, filtered, set of credentials
+   */
+  public static Credentials filterTokens(Credentials credentials,
+      List<Text> filter) {
+    Credentials result = new Credentials(credentials);
+    Iterator<Token<? extends TokenIdentifier>> iter =
+        result.getAllTokens().iterator();
+    while (iter.hasNext()) {
+      Token<? extends TokenIdentifier> token = iter.next();
+      LOG.debug("Token {}", token.getKind());
+      if (filter.contains(token.getKind())) {
+        LOG.debug("Filtering token {}", token.getKind());
+        iter.remove();
+      }
+    }
+    return result;
+  }
+
+  public static String dumpTokens(Credentials credentials, String separator) {
+    ArrayList<Token<? extends TokenIdentifier>> sorted =
+        new ArrayList<>(credentials.getAllTokens());
+    Collections.sort(sorted, new TokenComparator());
+    StringBuilder buffer = new StringBuilder(sorted.size()* 128);
+    for (Token<? extends TokenIdentifier> token : sorted) {
+      buffer.append(tokenToString(token)).append(separator);
+    }
+    return buffer.toString();
+  }
+
+  /**
+   * Create a string for people to look at
+   * @param token token to convert to a string form
+   * @return a printable view of the token
+   */
+  public static String tokenToString(Token<? extends TokenIdentifier> token) {
+    DateFormat df = DateFormat.getDateTimeInstance(
+        DateFormat.SHORT, DateFormat.SHORT);
+    StringBuilder buffer = new StringBuilder(128);
+    buffer.append(token.toString());
+    try {
+      TokenIdentifier ti = token.decodeIdentifier();
+      buffer.append("; ").append(ti);
+      if (ti instanceof AbstractDelegationTokenIdentifier) {
+        // details in human readable form, and compensate for information HDFS DT omits
+        AbstractDelegationTokenIdentifier dt = (AbstractDelegationTokenIdentifier) ti;
+        buffer.append("; Renewer: ").append(dt.getRenewer());
+        buffer.append("; Issued: ")
+            .append(df.format(new Date(dt.getIssueDate())));
+        buffer.append("; Max Date: ")
+            .append(df.format(new Date(dt.getMaxDate())));
+      }
+    } catch (IOException e) {
+      //marshall problem; not ours
+      LOG.debug("Failed to decode {}: {}", token, e, e);
+    }
+    return buffer.toString();
+  }
+
+  /**
+   * Get the expiry time of a token.
+   * @param token token to examine
+   * @return the time in milliseconds after which the token is invalid.
+   * @throws IOException
+   */
+  public static long getTokenExpiryTime(Token token) throws IOException {
+    TokenIdentifier identifier = token.decodeIdentifier();
+    Preconditions.checkState(identifier instanceof AbstractDelegationTokenIdentifier,
+        "Token %s of type: %s has an identifier which cannot be examined: %s",
+        token, token.getClass(), identifier);
+    AbstractDelegationTokenIdentifier id =
+        (AbstractDelegationTokenIdentifier) identifier;
+    return id.getMaxDate();
+  }
+
+  private static class TokenComparator
+      implements Comparator<Token<? extends TokenIdentifier>>, Serializable {
+    @Override
+    public int compare(Token<? extends TokenIdentifier> left,
+        Token<? extends TokenIdentifier> right) {
+      return left.getKind().toString().compareTo(right.getKind().toString());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/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
----------------------------------------------------------------------
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/slider/core/launch/JavaCommandLineBuilder.java
new file mode 100644
index 0000000..b8aa4c6
--- /dev/null
+++ b/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
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 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 java.util.Iterator;
+import java.util.Map;
+
+/**
+ * Command line builder purely for the Java CLI.
+ * Some of the <code>define</code> methods are designed to work with Hadoop tool and
+ * Slider launcher applications.
+ */
+public class JavaCommandLineBuilder extends CommandLineBuilder {
+
+  public JavaCommandLineBuilder() {
+    add(getJavaBinary());
+  }
+
+  /**
+   * Get the java binary. This is called in the constructor so don't try and
+   * do anything other than return a constant.
+   * @return the path to the Java binary
+   */
+  protected String getJavaBinary() {
+    return ApplicationConstants.Environment.JAVA_HOME.$$() + "/bin/java";
+  }
+
+  /**
+   * Set the size of the heap if a non-empty heap is passed in. 
+   * @param heap empty string or something like "128M" ,"1G" etc. The value is
+   * trimmed.
+   */
+  public void setJVMHeap(String heap) {
+    if (SliderUtils.isSet(heap)) {
+      add("-Xmx" + heap.trim());
+    }
+  }
+
+  /**
+   * Turn Java assertions on
+   */
+  public void enableJavaAssertions() {
+    add("-ea");
+    add("-esa");
+  }
+
+  /**
+   * Add a system property definition -must be used before setting the main entry point
+   * @param property
+   * @param value
+   */
+  public void sysprop(String property, String value) {
+    Preconditions.checkArgument(property != null, "null property name");
+    Preconditions.checkArgument(value != null, "null value");
+    add("-D" + property + "=" + value);
+  }
+  
+  public JavaCommandLineBuilder forceIPv4() {
+    sysprop("java.net.preferIPv4Stack", "true");
+    return this;
+  }
+  
+  public JavaCommandLineBuilder headless() {
+    sysprop("java.awt.headless", "true");
+    return this;
+  }
+
+  public boolean addConfOption(Configuration conf, String key) {
+    return defineIfSet(key, conf.get(key));
+  }
+
+  /**
+   * Add a varargs list of configuration parameters \u2014if they are present
+   * @param conf configuration source
+   * @param keys keys
+   */
+  public void addConfOptions(Configuration conf, String... keys) {
+    for (String key : keys) {
+      addConfOption(conf, key);
+    }
+  }
+
+  /**
+   * Add all configuration options which match the prefix
+   * @param conf configuration
+   * @param prefix prefix, e.g {@code "slider."}
+   * @return the number of entries copied
+   */
+  public int addPrefixedConfOptions(Configuration conf, String prefix) {
+    int copied = 0;
+    for (Map.Entry<String, String> entry : conf) {
+      if (entry.getKey().startsWith(prefix)) {
+        define(entry.getKey(), entry.getValue());
+        copied++;
+      }
+    }
+    return copied;
+  }
+
+  /**
+   * Ass a configuration option to the command line of  the application
+   * @param conf configuration
+   * @param key key
+   * @param defVal default value
+   * @return the resolved configuration option
+   * @throws IllegalArgumentException if key is null or the looked up value
+   * is null (that is: the argument is missing and devVal was null.
+   */
+  public String addConfOptionToCLI(Configuration conf,
+      String key,
+      String defVal) {
+    Preconditions.checkArgument(key != null, "null key");
+    String val = conf.get(key, defVal);
+    define(key, val);
+    return val;
+  }
+
+  /**
+   * Add a <code>-D key=val</code> command to the CLI. This is very Hadoop API
+   * @param key key
+   * @param val value
+   * @throws IllegalArgumentException if either argument is null
+   */
+  public void define(String key, String val) {
+    Preconditions.checkArgument(key != null, "null key");
+    Preconditions.checkArgument(val != null, "null value");
+    add("-D", key + "=" + val);
+  }
+
+  /**
+   * Add a <code>-D key=val</code> command to the CLI if <code>val</code>
+   * is not null
+   * @param key key
+   * @param val value
+   */
+  public boolean defineIfSet(String key, String val) {
+    Preconditions.checkArgument(key != null, "null key");
+    if (val != null) {
+      define(key, val);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Add a mandatory config option
+   * @param conf configuration
+   * @param key key
+   * @throws BadConfigException if the key is missing
+   */
+  public void addMandatoryConfOption(Configuration conf,
+      String key) throws BadConfigException {
+    if (!addConfOption(conf, key)) {
+      throw new BadConfigException("Missing configuration option: " + key);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/LaunchedApplication.java
----------------------------------------------------------------------
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/LaunchedApplication.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/LaunchedApplication.java
new file mode 100644
index 0000000..632e3fd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/LaunchedApplication.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.yarn.api.protocolrecords.KillApplicationResponse;
+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.slider.client.SliderYarnClientImpl;
+import org.apache.slider.common.tools.Duration;
+
+import java.io.IOException;
+
+/**
+ * Launched App with logic around it.
+ */
+public class LaunchedApplication {
+
+  protected final ApplicationId applicationId;
+  protected final SliderYarnClientImpl yarnClient;
+
+  public LaunchedApplication(ApplicationId applicationId,
+                             SliderYarnClientImpl yarnClient) {
+    assert applicationId != null;
+    assert yarnClient != null;
+    this.applicationId = applicationId;
+    this.yarnClient = yarnClient;
+  }
+
+  public LaunchedApplication(SliderYarnClientImpl yarnClient,
+                             ApplicationReport report) {
+    this.yarnClient = yarnClient;
+    this.applicationId = report.getApplicationId();
+  }
+
+  public ApplicationId getApplicationId() {
+    return applicationId;
+  }
+
+  /**
+   * 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 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(YarnApplicationState desiredState, Duration duration)
+    throws YarnException, IOException {
+    return yarnClient.monitorAppToState(applicationId, desiredState, duration);
+  }
+
+  /**
+   * Kill the submitted application by sending a call to the ASM
+   * @throws YarnException
+   * @throws IOException
+   */
+  public boolean forceKill(String reason)
+    throws YarnException, IOException {
+    if (applicationId != null) {
+      yarnClient.killRunningApplication(applicationId, reason);
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Kill the application
+   * @return the response
+   * @throws YarnException YARN problems
+   * @throws IOException IO problems
+   */
+  public KillApplicationResponse kill(String reason) throws
+                                                     YarnException,
+                                                     IOException {
+    return yarnClient.killRunningApplication(applicationId, reason);
+  }
+
+  /**
+   * Get the application report of this application
+   * @return an application report
+   * @throws YarnException
+   * @throws IOException
+   */
+  public ApplicationReport getApplicationReport()
+    throws YarnException, IOException {
+    return yarnClient.getApplicationReport(applicationId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/RunningApplication.java
----------------------------------------------------------------------
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/RunningApplication.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/RunningApplication.java
new file mode 100644
index 0000000..14c522c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/RunningApplication.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.slider.api.SliderClusterProtocol;
+import org.apache.slider.client.SliderYarnClientImpl;
+import org.apache.slider.common.SliderExitCodes;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.server.appmaster.rpc.RpcBinder;
+
+import java.io.IOException;
+
+import static org.apache.slider.common.Constants.CONNECT_TIMEOUT;
+import static org.apache.slider.common.Constants.RPC_TIMEOUT;
+
+/**
+ * A running application built from an app report. This one
+ * can be talked to
+ */
+public class RunningApplication extends LaunchedApplication {
+
+  private final ApplicationReport applicationReport;
+  public RunningApplication(SliderYarnClientImpl yarnClient,
+                            ApplicationReport applicationReport) {
+    super(yarnClient, applicationReport);
+    this.applicationReport = applicationReport;
+  }
+
+  public ApplicationReport getApplicationReport() {
+    return applicationReport;
+  }
+
+
+  /**
+   * Connect to a Slider AM
+   * @param app application report providing the details on the application
+   * @return an instance
+   * @throws YarnException
+   * @throws IOException
+   */
+  public SliderClusterProtocol connect(ApplicationReport app) throws
+                                                             YarnException,
+                                                             IOException {
+
+    try {
+      return RpcBinder.getProxy(yarnClient.getConfig(),
+                                yarnClient.getRmClient(),
+                                app,
+                                CONNECT_TIMEOUT,
+                                RPC_TIMEOUT);
+    } catch (InterruptedException e) {
+      throw new SliderException(SliderExitCodes.EXIT_TIMED_OUT,
+          e,
+          "Interrupted waiting for communications with the Application Master");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/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
----------------------------------------------------------------------
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/slider/core/launch/SerializedApplicationReport.java
new file mode 100644
index 0000000..8e0ef5a
--- /dev/null
+++ b/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
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+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.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.io.IOException;
+
+/**
+ * Serialized form of an application report which can be persisted
+ * and then parsed. It can not be converted back into a
+ * real YARN application report
+ * 
+ * Useful for testing
+ */
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+
+public class SerializedApplicationReport {
+
+  public String applicationId;
+  public String applicationAttemptId;
+  public String name;
+  public String applicationType;
+  public String user;
+  public String queue;
+  public String host;
+  public Integer rpcPort;
+  public String state;
+  public String diagnostics;
+  public String url;
+  /**
+   * This value is non-null only when a report is generated from a submission context.
+   * The YARN {@link ApplicationReport} structure does not propagate this value
+   * from the RM.
+   */
+  public Long submitTime;
+  public Long startTime;
+  public Long finishTime;
+  public String finalStatus;
+  public String origTrackingUrl;
+  public Float progress;
+  
+  public SerializedApplicationReport() {
+  }
+  
+  public SerializedApplicationReport(ApplicationReport report) {
+    this.applicationId = report.getApplicationId().toString();
+    ApplicationAttemptId attemptId = report.getCurrentApplicationAttemptId();
+    this.applicationAttemptId = attemptId != null ? attemptId.toString() : "N/A";
+    this.name = report.getName();
+    this.applicationType = report.getApplicationType();
+    this.user = report.getUser();
+    this.queue = report.getQueue();
+    this.host = report.getHost();
+    this.rpcPort = report.getRpcPort();
+    this.state = report.getYarnApplicationState().toString();
+    this.diagnostics = report.getDiagnostics();
+    this.startTime = report.getStartTime();
+    this.finishTime = report.getFinishTime();
+    FinalApplicationStatus appStatus = report.getFinalApplicationStatus();
+    this.finalStatus = appStatus == null ? "" : appStatus.toString();
+    this.progress = report.getProgress();
+    this.url = report.getTrackingUrl();
+    this.origTrackingUrl= report.getOriginalTrackingUrl();
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return ApplicationReportSerDeser.toString(this);
+    } catch (IOException e) {
+      return super.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/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
----------------------------------------------------------------------
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/slider/core/main/ExitCodeProvider.java
new file mode 100644
index 0000000..503b9b9
--- /dev/null
+++ b/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
@@ -0,0 +1,32 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+/**
+ * Get the exit code of an exception. Making it an interface allows
+ * us to retrofit exit codes onto existing classes
+ */
+public interface ExitCodeProvider {
+
+  /**
+   * Method to get the exit code
+   * @return the exit code
+   */
+  int  getExitCode();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..42442d1
--- /dev/null
+++ 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
@@ -0,0 +1,103 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/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
----------------------------------------------------------------------
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/slider/core/main/LauncherExitCodes.java
new file mode 100644
index 0000000..83e89f0
--- /dev/null
+++ b/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
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+/*
+ * Common Exit codes
+ * <p>
+ * Exit codes from 64 up are application specific.
+ * <p>
+ * Many of the exit codes are designed to resemble HTTP error codes,
+ * squashed into a single byte. e.g 44 , "not found" is the equivalent
+ * of 404
+ * <pre>
+ *    0-10: general command issues
+ *   30-39: equivalent to the 3XX responses, where those responses are
+ *          considered errors by the application.
+ *   40-49: request-related errors
+ *   50-59: server-side problems. These may be triggered by the request.
+ *   64-  : application specific error codes
+ * </pre>
+ */
+public interface LauncherExitCodes {
+  
+  /**
+   * 0: success
+   */
+  int EXIT_SUCCESS                    =  0;
+
+  /**
+   * -1: generic "false" response. The operation worked but
+   * the result was not true
+   */
+  int EXIT_FALSE                      = -1;
+
+  /**
+   * Exit code when a client requested service termination: {@value}
+   */
+  int EXIT_CLIENT_INITIATED_SHUTDOWN  =  1;
+
+  /**
+   * Exit code when targets could not be launched: {@value}
+   */
+  int EXIT_TASK_LAUNCH_FAILURE        =  2;
+
+  /**
+   * Exit code when a control-C, kill -3, signal was picked up: {@value}
+   */
+  int EXIT_INTERRUPTED                = 3;
+
+  /**
+   * Exit code when a usage message was printed: {@value}
+   */
+  int EXIT_USAGE                      = 4;
+
+  /**
+   * Exit code when something happened but we can't be specific: {@value}
+   */
+  int EXIT_OTHER_FAILURE               = 5;
+
+  /**
+   * Exit code on connectivity problems: {@value}
+   */
+  int EXIT_MOVED                      = 31;
+  
+  /**
+   * found: {@value}.
+   * <p>
+   * This is low value as in HTTP it is normally a success/redirect;
+   * whereas on the command line 0 is the sole success code.
+   * <p>
+   * <code>302 Found</code>
+   */
+  int EXIT_FOUND                      = 32;
+
+  /**
+   * Exit code on a request where the destination has not changed
+   * and (somehow) the command specified that this is an error.
+   * That is, this exit code is somehow different from a "success"
+   * : {@value}
+   * <p>
+   * <code>304 Not Modified </code>
+  */
+  int EXIT_NOT_MODIFIED               = 34;
+
+  /**
+   * Exit code when the command line doesn't parse: {@value}, or
+   * when it is otherwise invalid.
+   * <p>
+   * <code>400 BAD REQUEST</code>
+   */
+  int EXIT_COMMAND_ARGUMENT_ERROR     = 40;
+
+  /**
+   * The request requires user authentication: {@value}
+   * <p>
+   * <code>401 Unauthorized</code>
+   */
+  int EXIT_UNAUTHORIZED               = 41;
+  
+  /**
+   * Forbidden action: {@value}
+   * <p>
+   * <code>403: Forbidden</code>
+   */
+  int EXIT_FORBIDDEN                  = 43;
+  
+  /**
+   * Something was not found: {@value}
+   * <p>
+   * <code>404: NOT FOUND</code>
+   */
+  int EXIT_NOT_FOUND                  = 44;
+
+  /**
+   * The operation is not allowed: {@value}
+   * <p>
+   * <code>405: NOT ALLOWED</code>
+   */
+  int EXIT_OPERATION_NOT_ALLOWED       = 45;
+
+  /**
+   * The command is somehow not acceptable: {@value}
+   * <p>
+   * <code>406: NOT ACCEPTABLE</code>
+   */
+  int EXIT_NOT_ACCEPTABLE            = 46;
+
+  /**
+   * Exit code on connectivity problems: {@value}
+   * <p>
+   * <code>408: Request Timeout</code>
+   */
+  int EXIT_CONNECTIVITY_PROBLEM       = 48;
+
+  /**
+   * The request could not be completed due to a conflict with the current
+   * state of the resource.  {@value}
+   * <p>
+   * <code>409: conflict</code>
+   */
+  int EXIT_CONFLICT                   = 49;
+
+  /**
+   * internal error: {@value}
+   * <p>
+   * <code>500 Internal Server Error</code>
+   */
+  int EXIT_INTERNAL_ERROR             = 50;
+
+  /**
+   * Unimplemented feature: {@value}
+   * <p>
+   * <code>501: Not Implemented</code>
+   */
+  int EXIT_UNIMPLEMENTED              = 51;
+
+  /**
+   * Service Unavailable; it may be available later: {@value}
+   * <p>
+   * <code>503 Service Unavailable</code>
+   */
+  int EXIT_SERVICE_UNAVAILABLE        = 53;
+
+  /**
+   * The application does not support, or refuses to support this version: {@value}.
+   * If raised, this is expected to be raised server-side and likely due
+   * to client/server version incompatibilities.
+   * <p>
+   * <code> 505: Version Not Supported</code>
+   */
+  int EXIT_UNSUPPORTED_VERSION        = 55;
+
+  /**
+   * Exit code when an exception was thrown from the service: {@value}
+   * <p>
+   * <code>5XX</code>
+   */
+  int EXIT_EXCEPTION_THROWN           = 56;
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org