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 2017/09/25 23:39:03 UTC

[64/86] [abbrv] hadoop git commit: YARN-7091. Rename application to service in yarn-native-services. Contributed by Jian He

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CommandLineBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CommandLineBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CommandLineBuilder.java
new file mode 100644
index 0000000..7baa284
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CommandLineBuilder.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.containerlaunch;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.service.utils.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 number of arguments
+  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, " ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
new file mode 100644
index 0000000..0e51a62
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.containerlaunch;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.service.api.records.Component;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
+import org.apache.hadoop.yarn.service.provider.ProviderService;
+import org.apache.hadoop.yarn.service.provider.ProviderFactory;
+import org.apache.hadoop.yarn.service.api.records.Service;
+import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+public class ContainerLaunchService extends AbstractService{
+
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(ContainerLaunchService.class);
+
+  private ExecutorService executorService;
+  private SliderFileSystem fs;
+
+  public ContainerLaunchService(SliderFileSystem fs) {
+    super(ContainerLaunchService.class.getName());
+    this.fs = fs;
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    executorService = Executors.newCachedThreadPool();
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (executorService != null) {
+      executorService.shutdownNow();
+    }
+    super.serviceStop();
+  }
+
+  public void launchCompInstance(Service service,
+      ComponentInstance instance, Container container) {
+    ContainerLauncher launcher =
+        new ContainerLauncher(service, instance, container);
+    executorService.execute(launcher);
+  }
+
+  private class ContainerLauncher implements Runnable {
+    public final Container container;
+    public final Service service;
+    public ComponentInstance instance;
+
+    public ContainerLauncher(
+        Service service,
+        ComponentInstance instance, Container container) {
+      this.container = container;
+      this.service = service;
+      this.instance = instance;
+    }
+
+    @Override public void run() {
+      Component compSpec = instance.getCompSpec();
+      ProviderService provider = ProviderFactory.getProviderService(
+          compSpec.getArtifact());
+      AbstractLauncher launcher = new AbstractLauncher(fs, null);
+      try {
+        provider.buildContainerLaunchContext(launcher, service,
+            instance, fs, getConfig());
+        instance.getComponent().getScheduler().getNmClient()
+            .startContainerAsync(container,
+                launcher.completeContainerLaunch());
+      } catch (Exception e) {
+        LOG.error(instance.getCompInstanceId()
+            + ": Failed to launch container. ", e);
+
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CredentialUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CredentialUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CredentialUtils.java
new file mode 100644
index 0000000..fce58e5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CredentialUtils.java
@@ -0,0 +1,319 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.containerlaunch;
+
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataOutputStream;
+import java.io.File;
+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 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 {
+
+  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;
+  }
+
+  /**
+   * 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/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/JavaCommandLineBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/JavaCommandLineBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/JavaCommandLineBuilder.java
new file mode 100644
index 0000000..cbcb0d6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/JavaCommandLineBuilder.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.containerlaunch;
+
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.service.utils.SliderUtils;
+import org.apache.hadoop.yarn.service.exceptions.BadConfigException;
+
+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 —if 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/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadClusterStateException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadClusterStateException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadClusterStateException.java
new file mode 100644
index 0000000..db9de7a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadClusterStateException.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.exceptions;
+
+import org.apache.hadoop.yarn.service.exceptions.SliderException;
+
+/**
+ * The system is in a bad state
+ */
+public class BadClusterStateException extends SliderException {
+  public BadClusterStateException(String message,
+                                  Object... args) {
+    super(EXIT_BAD_STATE, message, args);
+  }
+
+  public BadClusterStateException(Throwable throwable,
+                                  String message, Object... args) {
+    super(EXIT_BAD_STATE, throwable, message, args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadCommandArgumentsException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadCommandArgumentsException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadCommandArgumentsException.java
new file mode 100644
index 0000000..41e3251
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadCommandArgumentsException.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.service.exceptions;
+
+public class BadCommandArgumentsException extends SliderException {
+  public BadCommandArgumentsException(String s, Object... args) {
+    super(EXIT_COMMAND_ARGUMENT_ERROR, s, args);
+  }
+
+  public BadCommandArgumentsException(Throwable throwable, String message,
+                                      Object... args) {
+    super(EXIT_COMMAND_ARGUMENT_ERROR, throwable, message, args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadConfigException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadConfigException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadConfigException.java
new file mode 100644
index 0000000..8199c3c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/BadConfigException.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.service.exceptions;
+
+/**
+ * An exception to raise on a bad configuration
+ */
+public class BadConfigException extends SliderException {
+
+  public BadConfigException(String s) {
+    super(EXIT_BAD_CONFIGURATION, s);
+  }
+
+  public BadConfigException(String message, Object... args) {
+    super(EXIT_BAD_CONFIGURATION, message, args);
+  }
+
+  public BadConfigException(
+                            Throwable throwable,
+                            String message, Object... args) {
+    super(EXIT_BAD_CONFIGURATION, throwable, message, args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ErrorStrings.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ErrorStrings.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ErrorStrings.java
new file mode 100644
index 0000000..83658c8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ErrorStrings.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.exceptions;
+
+public interface ErrorStrings {
+
+  String PRINTF_E_INSTANCE_ALREADY_EXISTS = "Service Instance \"%s\" already exists and is defined in %s";
+  String PRINTF_E_INSTANCE_DIR_ALREADY_EXISTS = "Service Instance dir already exists: %s";
+
+  /**
+   * ERROR Strings
+   */
+  String ERROR_NO_ACTION = "No action specified";
+  String ERROR_UNKNOWN_ACTION = "Unknown command: ";
+  String ERROR_NOT_ENOUGH_ARGUMENTS =
+    "Not enough arguments for action: ";
+  String ERROR_PARSE_FAILURE =
+      "Failed to parse ";
+  /**
+   * All the remaining values after argument processing
+   */
+  String ERROR_TOO_MANY_ARGUMENTS =
+    "Too many arguments";
+  String ERROR_DUPLICATE_ENTRY = "Duplicate entry for ";
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ExitCodeProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ExitCodeProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ExitCodeProvider.java
new file mode 100644
index 0000000..d66b860
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/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.hadoop.yarn.service.exceptions;
+
+/**
+ * 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/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/LauncherExitCodes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/LauncherExitCodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/LauncherExitCodes.java
new file mode 100644
index 0000000..483fb48
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/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.hadoop.yarn.service.exceptions;
+
+/*
+ * Common Exit codes
+ * <p>
+ * Exit codes from 64 up are service 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 service.
+ *   40-49: request-related errors
+ *   50-59: server-side problems. These may be triggered by the request.
+ *   64-  : service 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 service 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;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
new file mode 100644
index 0000000..ef22b57
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.exceptions;
+
+public interface RestApiErrorMessages {
+  String ERROR_APPLICATION_NAME_INVALID =
+      "Service name is either empty or not provided";
+  String ERROR_APPLICATION_NAME_INVALID_FORMAT =
+      "Service name %s is not valid - only lower case letters, digits, " +
+          "and hyphen are allowed, and the name must be no more " +
+          "than 63 characters";
+  String ERROR_COMPONENT_NAME_INVALID =
+      "Component name must be no more than %s characters: %s";
+  String ERROR_USER_NAME_INVALID =
+      "User name must be no more than 63 characters";
+
+  String ERROR_APPLICATION_NOT_RUNNING = "Service not running";
+  String ERROR_APPLICATION_DOES_NOT_EXIST = "Service not found";
+  String ERROR_APPLICATION_IN_USE = "Service already exists in started"
+      + " state";
+  String ERROR_APPLICATION_INSTANCE_EXISTS = "Service already exists in"
+      + " stopped/failed state (either restart with PUT or destroy with DELETE"
+      + " before creating a new one)";
+
+  String ERROR_SUFFIX_FOR_COMPONENT =
+      " for component %s (nor at the global level)";
+  String ERROR_ARTIFACT_INVALID = "Artifact is not provided";
+  String ERROR_ARTIFACT_FOR_COMP_INVALID =
+      ERROR_ARTIFACT_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_ARTIFACT_ID_INVALID =
+      "Artifact id (like docker image name) is either empty or not provided";
+  String ERROR_ARTIFACT_ID_FOR_COMP_INVALID =
+      ERROR_ARTIFACT_ID_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+
+  String ERROR_RESOURCE_INVALID = "Resource is not provided";
+  String ERROR_RESOURCE_FOR_COMP_INVALID =
+      ERROR_RESOURCE_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_RESOURCE_MEMORY_INVALID =
+      "Service resource or memory not provided";
+  String ERROR_RESOURCE_CPUS_INVALID =
+      "Service resource or cpus not provided";
+  String ERROR_RESOURCE_CPUS_INVALID_RANGE =
+      "Unacceptable no of cpus specified, either zero or negative";
+  String ERROR_RESOURCE_MEMORY_FOR_COMP_INVALID =
+      ERROR_RESOURCE_MEMORY_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_RESOURCE_CPUS_FOR_COMP_INVALID =
+      ERROR_RESOURCE_CPUS_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_RESOURCE_CPUS_FOR_COMP_INVALID_RANGE =
+      ERROR_RESOURCE_CPUS_INVALID_RANGE
+          + " for component %s (or at the global level)";
+  String ERROR_CONTAINERS_COUNT_INVALID =
+      "Invalid no of containers specified";
+  String ERROR_CONTAINERS_COUNT_FOR_COMP_INVALID =
+      ERROR_CONTAINERS_COUNT_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_DEPENDENCY_INVALID = "Dependency %s for component %s is " +
+      "invalid, does not exist as a component";
+  String ERROR_DEPENDENCY_CYCLE = "Invalid dependencies, a cycle may " +
+      "exist: %s";
+
+  String ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_NOT_SUPPORTED =
+      "Cannot specify" + " cpus/memory along with profile";
+  String ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_FOR_COMP_NOT_SUPPORTED =
+      ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_NOT_SUPPORTED
+          + " for component %s";
+  String ERROR_RESOURCE_PROFILE_NOT_SUPPORTED_YET =
+      "Resource profile is not " + "supported yet. Please specify cpus/memory.";
+
+  String ERROR_NULL_ARTIFACT_ID =
+      "Artifact Id can not be null if artifact type is none";
+  String ERROR_ABSENT_NUM_OF_INSTANCE =
+      "Num of instances should appear either globally or per component";
+  String ERROR_ABSENT_LAUNCH_COMMAND =
+      "Launch_command is required when type is not DOCKER";
+
+  String ERROR_QUICKLINKS_FOR_COMP_INVALID = "Quicklinks specified at"
+      + " component level, needs corresponding values set at service level";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ServiceLaunchException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ServiceLaunchException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ServiceLaunchException.java
new file mode 100644
index 0000000..e83ccbe
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/ServiceLaunchException.java
@@ -0,0 +1,73 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS 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.exceptions;
+
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * A service launch exception that includes an exit code;
+ * when caught by the ServiceLauncher, it will convert that
+ * into a process exit code.
+ */
+public class ServiceLaunchException extends YarnException
+  implements ExitCodeProvider, LauncherExitCodes {
+
+  private final int exitCode;
+
+  /**
+   * Create an exception with the specific exit code
+   * @param exitCode exit code
+   * @param cause cause of the exception
+   */
+  public ServiceLaunchException(int exitCode, Throwable cause) {
+    super(cause);
+    this.exitCode = exitCode;
+  }
+
+  /**
+   * Create an exception with the specific exit code and text
+   * @param exitCode exit code
+   * @param message message to use in exception
+   */
+  public ServiceLaunchException(int exitCode, String message) {
+    super(message);
+    this.exitCode = exitCode;
+  }
+
+  /**
+   * Create an exception with the specific exit code, text and cause
+   * @param exitCode exit code
+   * @param message message to use in exception
+   * @param cause cause of the exception
+   */
+  public ServiceLaunchException(int exitCode, String message, Throwable cause) {
+    super(message, cause);
+    this.exitCode = exitCode;
+  }
+
+  /**
+   * Get the exit code
+   * @return the exit code
+   */
+  @Override
+  public int getExitCode() {
+    return exitCode;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/SliderException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/SliderException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/SliderException.java
new file mode 100644
index 0000000..5b74b80
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/SliderException.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.service.exceptions;
+
+import org.apache.hadoop.yarn.service.conf.SliderExitCodes;
+
+public class SliderException extends ServiceLaunchException implements
+    SliderExitCodes {
+  public SliderException() {
+    super(EXIT_EXCEPTION_THROWN, "SliderException");
+  }
+
+  public SliderException(int code, String message) {
+    super(code, message);
+  }
+
+  public SliderException(String s) {
+    super(EXIT_EXCEPTION_THROWN, s);
+  }
+
+  public SliderException(String s, Throwable throwable) {
+    super(EXIT_EXCEPTION_THROWN, s, throwable);
+  }
+
+  /**
+   * Format the exception as you create it
+   * @param code exit code
+   * @param message exception message -sprintf formatted
+   * @param args arguments for the formatting
+   */
+  public SliderException(int code, String message, Object... args) {
+    super(code, String.format(message, args));
+  }
+
+  /**
+   * Format the exception, include a throwable. 
+   * The throwable comes before the message so that it is out of the varargs
+   * @param code exit code
+   * @param throwable thrown
+   * @param message message
+   * @param args arguments
+   */
+  public SliderException(int code,
+      Throwable throwable,
+      String message,
+      Object... args) {
+    super(code, String.format(message, args), throwable);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/UsageException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/UsageException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/UsageException.java
new file mode 100644
index 0000000..3a9fa25
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/UsageException.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.service.exceptions;
+
+/**
+ * Used to raise a usage exception ... this has the exit code
+ * {@link #EXIT_USAGE}
+ */
+public class UsageException extends SliderException {
+  public UsageException(String s, Object... args) {
+    super(EXIT_USAGE, s, args);
+  }
+
+  public UsageException(Throwable throwable, String message,
+      Object... args) {
+    super(EXIT_USAGE, throwable, message, args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/impl/pb/client/ClientAMProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/impl/pb/client/ClientAMProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/impl/pb/client/ClientAMProtocolPBClientImpl.java
new file mode 100644
index 0000000..33e33a6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/impl/pb/client/ClientAMProtocolPBClientImpl.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.impl.pb.client;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
+import org.apache.hadoop.yarn.service.ClientAMProtocol;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsResponseProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto;
+import org.apache.hadoop.yarn.service.impl.pb.service.ClientAMProtocolPB;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopResponseProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopRequestProto;
+
+public class ClientAMProtocolPBClientImpl
+    implements ClientAMProtocol, Closeable {
+
+  private ClientAMProtocolPB proxy;
+
+  public ClientAMProtocolPBClientImpl(long clientVersion,
+      InetSocketAddress addr, Configuration conf) throws IOException {
+    RPC.setProtocolEngine(conf, ClientAMProtocolPB.class,
+        ProtobufRpcEngine.class);
+    proxy = RPC.getProxy(ClientAMProtocolPB.class, clientVersion, addr, conf);
+
+  }
+
+  @Override public FlexComponentsResponseProto flexComponents(
+      FlexComponentsRequestProto request) throws IOException, YarnException {
+    try {
+      return proxy.flexComponents(null, request);
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+    }
+    return null;
+  }
+
+  @Override
+  public GetStatusResponseProto getStatus(GetStatusRequestProto request)
+      throws IOException, YarnException {
+    try {
+      return proxy.getStatus(null, request);
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+    }
+    return null;
+  }
+
+  @Override
+  public StopResponseProto stop(StopRequestProto requestProto)
+      throws IOException, YarnException {
+    try {
+      return proxy.stop(null, requestProto);
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+    }
+    return null;
+  }
+
+  @Override public void close() {
+    if (this.proxy != null) {
+      RPC.stopProxy(this.proxy);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/impl/pb/service/ClientAMProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/impl/pb/service/ClientAMProtocolPB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/impl/pb/service/ClientAMProtocolPB.java
new file mode 100644
index 0000000..6a9cd37
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/impl/pb/service/ClientAMProtocolPB.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.impl.pb.service;
+
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol;
+
+@ProtocolInfo(
+    protocolName = "org.apache.hadoop.yarn.service.ClientAMProtocol",
+    protocolVersion = 1)
+public interface ClientAMProtocolPB extends
+    ClientAMProtocol.ClientAMProtocolService.BlockingInterface {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/impl/pb/service/ClientAMProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/impl/pb/service/ClientAMProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/impl/pb/service/ClientAMProtocolPBServiceImpl.java
new file mode 100644
index 0000000..7100781
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/impl/pb/service/ClientAMProtocolPBServiceImpl.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.service.impl.pb.service;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsResponseProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto;
+import org.apache.hadoop.yarn.service.ClientAMProtocol;
+
+import java.io.IOException;
+
+public class ClientAMProtocolPBServiceImpl implements ClientAMProtocolPB {
+
+  private ClientAMProtocol real;
+
+  public ClientAMProtocolPBServiceImpl(ClientAMProtocol impl) {
+    this.real = impl;
+  }
+
+  @Override
+  public FlexComponentsResponseProto flexComponents(RpcController controller,
+      FlexComponentsRequestProto request) throws ServiceException {
+    try {
+      return real.flexComponents(request);
+    } catch (IOException | YarnException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override public GetStatusResponseProto getStatus(RpcController controller,
+      GetStatusRequestProto request) throws ServiceException {
+    try {
+      return real.getStatus(request);
+    } catch (IOException | YarnException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public org.apache.hadoop.yarn.proto.ClientAMProtocol.StopResponseProto stop(
+      RpcController controller,
+      org.apache.hadoop.yarn.proto.ClientAMProtocol.StopRequestProto request)
+      throws ServiceException {
+    try {
+      return real.stop(request);
+    } catch (IOException | YarnException e) {
+      throw new ServiceException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/ServiceMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/ServiceMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/ServiceMonitor.java
new file mode 100644
index 0000000..982448a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/ServiceMonitor.java
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.monitor;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.service.ServiceContext;
+import org.apache.hadoop.yarn.service.component.Component;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
+import org.apache.hadoop.yarn.service.conf.YarnServiceConf;
+import org.apache.hadoop.yarn.service.component.ComponentEvent;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
+import org.apache.hadoop.yarn.service.component.ComponentState;
+import org.apache.hadoop.yarn.service.monitor.probe.ProbeStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.yarn.service.component.instance.ComponentInstanceState.STARTED;
+import static org.apache.hadoop.yarn.service.component.ComponentEventType.FLEX;
+import static org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType.BECOME_NOT_READY;
+import static org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType.BECOME_READY;
+import static org.apache.hadoop.yarn.service.component.instance.ComponentInstanceState.READY;
+import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.CONTAINER_FAILURE_WINDOW;
+import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.DEFAULT_READINESS_CHECK_INTERVAL;
+import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.READINESS_CHECK_INTERVAL;
+
+public class ServiceMonitor extends AbstractService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ServiceMonitor.class);
+
+  public ScheduledExecutorService executorService;
+  private  Map<ContainerId, ComponentInstance> liveInstances = null;
+  private ServiceContext context;
+  private Configuration conf;
+
+  public ServiceMonitor(String name, ServiceContext context) {
+    super(name);
+    liveInstances = context.scheduler.getLiveInstances();
+    this.context = context;
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    executorService = Executors.newScheduledThreadPool(1);
+    this.conf = conf;
+    super.serviceInit(conf);
+  }
+
+  @Override
+  public void serviceStart() throws Exception {
+    long readinessCheckInterval = YarnServiceConf
+        .getLong(READINESS_CHECK_INTERVAL, DEFAULT_READINESS_CHECK_INTERVAL,
+            context.service.getConfiguration(), conf);
+
+    executorService
+        .scheduleAtFixedRate(new ReadinessChecker(), readinessCheckInterval,
+            readinessCheckInterval, TimeUnit.SECONDS);
+
+    // Default 6 hours.
+    long failureResetInterval = YarnServiceConf
+        .getLong(CONTAINER_FAILURE_WINDOW, 21600,
+            context.service.getConfiguration(), conf);
+
+    executorService
+        .scheduleAtFixedRate(new ContainerFailureReset(), failureResetInterval,
+            failureResetInterval, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void serviceStop() throws Exception {
+    if (executorService != null) {
+      executorService.shutdownNow();
+    }
+  }
+
+  private class ReadinessChecker implements Runnable {
+
+    @Override
+    public void run() {
+
+      // check if the comp instance are ready
+      for (Map.Entry<ContainerId, ComponentInstance> entry : liveInstances
+          .entrySet()) {
+        ComponentInstance instance = entry.getValue();
+
+        ProbeStatus status = instance.ping();
+        if (status.isSuccess()) {
+          if (instance.getState() == STARTED) {
+            // synchronously update the state.
+            instance.handle(
+                new ComponentInstanceEvent(entry.getKey(), BECOME_READY));
+          }
+        } else {
+          if (instance.getState() == READY) {
+            instance.handle(
+                new ComponentInstanceEvent(entry.getKey(), BECOME_NOT_READY));
+          }
+        }
+      }
+
+      for (Component component : context.scheduler.getAllComponents()
+          .values()) {
+        // If comp hasn't started yet and its dependencies are satisfied
+        if (component.getState() == ComponentState.INIT && component
+            .areDependenciesReady()) {
+          LOG.info("[COMPONENT {}]: Dependencies satisfied, ramping up.",
+              component.getName());
+          ComponentEvent event = new ComponentEvent(component.getName(), FLEX)
+              .setDesired(component.getComponentSpec().getNumberOfContainers());
+          component.handle(event);
+        }
+      }
+    }
+  }
+
+  private class ContainerFailureReset implements Runnable {
+    @Override
+    public void run() {
+      for (Component component : context.scheduler.getAllComponents().values()) {
+        component.resetCompFailureCount();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/HttpProbe.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/HttpProbe.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/HttpProbe.java
new file mode 100644
index 0000000..1923086
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/HttpProbe.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.monitor.probe;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
+import org.apache.hadoop.yarn.service.utils.SliderUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.Map;
+
+public class HttpProbe extends Probe {
+  protected static final Logger log = LoggerFactory.getLogger(HttpProbe.class);
+
+  private static final String HOST_TOKEN = "${THIS_HOST}";
+
+  private final String urlString;
+  private final int timeout;
+  private final int min, max;
+
+
+  public HttpProbe(String url, int timeout, int min, int max, Configuration
+      conf) {
+    super("Http probe of " + url + " [" + min + "-" + max + "]", conf);
+    this.urlString = url;
+    this.timeout = timeout;
+    this.min = min;
+    this.max = max;
+  }
+
+  public static HttpProbe create(Map<String, String> props)
+      throws IOException {
+    String urlString = getProperty(props, WEB_PROBE_URL, null);
+    new URL(urlString);
+    int timeout = getPropertyInt(props, WEB_PROBE_CONNECT_TIMEOUT,
+        WEB_PROBE_CONNECT_TIMEOUT_DEFAULT);
+    int minSuccess = getPropertyInt(props, WEB_PROBE_MIN_SUCCESS,
+        WEB_PROBE_MIN_SUCCESS_DEFAULT);
+    int maxSuccess = getPropertyInt(props, WEB_PROBE_MAX_SUCCESS,
+        WEB_PROBE_MAX_SUCCESS_DEFAULT);
+    return new HttpProbe(urlString, timeout, minSuccess, maxSuccess, null);
+  }
+
+
+  private static HttpURLConnection getConnection(URL url, int timeout) throws
+      IOException {
+    HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+    connection.setInstanceFollowRedirects(true);
+    connection.setConnectTimeout(timeout);
+    return connection;
+  }
+
+  @Override
+  public ProbeStatus ping(ComponentInstance instance) {
+    ProbeStatus status = new ProbeStatus();
+    ContainerStatus containerStatus = instance.getContainerStatus();
+    if (containerStatus == null || SliderUtils.isEmpty(containerStatus.getIPs())
+        || StringUtils.isEmpty(containerStatus.getHost())) {
+      status.fail(this, new IOException("IP is not available yet"));
+      return status;
+    }
+
+    String ip = containerStatus.getIPs().get(0);
+    HttpURLConnection connection = null;
+    try {
+      URL url = new URL(urlString.replace(HOST_TOKEN, ip));
+      connection = getConnection(url, this.timeout);
+      int rc = connection.getResponseCode();
+      if (rc < min || rc > max) {
+        String error = "Probe " + url + " error code: " + rc;
+        log.info(error);
+        status.fail(this,
+            new IOException(error));
+      } else {
+        status.succeed(this);
+      }
+    } catch (Throwable e) {
+      String error = "Probe " + urlString + " failed for IP " + ip + ": " + e;
+      log.info(error, e);
+      status.fail(this,
+          new IOException(error, e));
+    } finally {
+      if (connection != null) {
+        connection.disconnect();
+      }
+    }
+    return status;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/LogEntryBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/LogEntryBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/LogEntryBuilder.java
new file mode 100644
index 0000000..9ad86fe
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/LogEntryBuilder.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.service.monitor.probe;
+
+/**
+ * Build up log entries for ease of splunk
+ */
+public class LogEntryBuilder {
+
+  private final StringBuilder builder = new StringBuilder();
+
+  public LogEntryBuilder() {
+  }
+
+  public LogEntryBuilder(String text) {
+    elt(text);
+  }
+
+
+  public LogEntryBuilder(String name, Object value) {
+    entry(name, value);
+  }
+
+  public LogEntryBuilder elt(String text) {
+    addComma();
+    builder.append(text);
+    return this;
+  }
+
+  public LogEntryBuilder elt(String name, Object value) {
+    addComma();
+    entry(name, value);
+    return this;
+  }
+
+  private void addComma() {
+    if (!isEmpty()) {
+      builder.append(", ");
+    }
+  }
+
+  private void entry(String name, Object value) {
+    builder.append(name).append('=');
+    if (value != null) {
+      builder.append('"').append(value.toString()).append('"');
+    } else {
+      builder.append("null");
+    }
+  }
+
+  @Override
+  public String toString() {
+    return builder.toString();
+  }
+
+  private boolean isEmpty() {
+    return builder.length() == 0;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/MonitorKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/MonitorKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/MonitorKeys.java
new file mode 100644
index 0000000..55b55f6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/MonitorKeys.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.service.monitor.probe;
+
+/**
+ * Config keys for monitoring
+ */
+public interface MonitorKeys {
+
+  /**
+   * Port probing key : port to attempt to create a TCP connection to {@value}.
+   */
+  String PORT_PROBE_PORT = "port";
+  /**
+   * Port probing key : timeout for the the connection attempt {@value}.
+   */
+  String PORT_PROBE_CONNECT_TIMEOUT = "timeout";
+  /**
+   * Port probing default : timeout for the connection attempt {@value}.
+   */
+  int PORT_PROBE_CONNECT_TIMEOUT_DEFAULT = 1000;
+
+  /**
+   * Web probing key : URL {@value}.
+   */
+  String WEB_PROBE_URL = "url";
+  /**
+   * Web probing key : min success code {@value}.
+   */
+  String WEB_PROBE_MIN_SUCCESS = "min.success";
+  /**
+   * Web probing key : max success code {@value}.
+   */
+  String WEB_PROBE_MAX_SUCCESS = "max.success";
+  /**
+   * Web probing default : min successful response code {@value}.
+   */
+  int WEB_PROBE_MIN_SUCCESS_DEFAULT = 200;
+  /**
+   * Web probing default : max successful response code {@value}.
+   */
+  int WEB_PROBE_MAX_SUCCESS_DEFAULT = 299;
+  /**
+   * Web probing key : timeout for the connection attempt {@value}
+   */
+  String WEB_PROBE_CONNECT_TIMEOUT = "timeout";
+  /**
+   * Port probing default : timeout for the connection attempt {@value}.
+   */
+  int WEB_PROBE_CONNECT_TIMEOUT_DEFAULT = 1000;
+}


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