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/13 22:53:17 UTC

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderVersionInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderVersionInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderVersionInfo.java
new file mode 100644
index 0000000..86025ee
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderVersionInfo.java
@@ -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.common.tools;
+
+import org.apache.hadoop.util.VersionInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.Locale;
+import java.util.Properties;
+
+/**
+ * Extract the version properties, which will look something like
+ * <pre>
+ * application.name=${pom.name}
+ * application.version=${pom.version}
+ * application.build=${buildNumber}
+ * application.build.java.version=${java.version}
+ * application.build.info=${pom.name}-${pom.version} Built against ${buildNumber} on ${java.version} by ${user.name}
+ * </pre>
+ * 
+ * the <code>mvn process-resources</code> target will expand the properties
+ * and add the resources to target/classes, which will then look something like
+ * <pre>
+ *   application.name=Slider Core
+ *   application.version=0.7.1-SNAPSHOT
+ *   application.build=1dd69
+ *   application.build.java.version=1.7.0_45
+ *   application.build.user=stevel
+ *   application.build.info=Slider Core-0.7.1-SNAPSHOT Built against 1dd69 on 1.7.0_45 by stevel
+ * </pre>
+ * 
+ * Note: the values will change and more properties added.
+ */
+public class SliderVersionInfo {
+  private static final Logger log = LoggerFactory.getLogger(SliderVersionInfo.class);
+
+  /**
+   * Name of the resource containing the filled-in-at-runtime props
+   */
+  public static final String VERSION_RESOURCE =
+      "org/apache/slider/providers/dynamic/application.properties";
+
+  public static final String APP_NAME = "application.name";
+  public static final String APP_VERSION = "application.version";
+  public static final String APP_BUILD = "application.build";
+  public static final String APP_BUILD_JAVA_VERSION = "application.build.java.version";
+  public static final String APP_BUILD_USER = "application.build.user";
+  public static final String APP_BUILD_INFO = "application.build.info";
+  public static final String HADOOP_BUILD_INFO = "hadoop.build.info";
+  public static final String HADOOP_DEPLOYED_INFO = "hadoop.deployed.info";
+
+
+  public static Properties loadVersionProperties()  {
+    Properties props = new Properties();
+    URL resURL = SliderVersionInfo.class.getClassLoader()
+                                   .getResource(VERSION_RESOURCE);
+    assert resURL != null : "Null resource " + VERSION_RESOURCE;
+
+    try {
+      InputStream inStream = resURL.openStream();
+      assert inStream != null : "Null input stream from " + VERSION_RESOURCE;
+      props.load(inStream);
+    } catch (IOException e) {
+      log.warn("IOE loading " + VERSION_RESOURCE, e);
+    }
+    return props;
+  }
+
+  /**
+   * Load the version info and print it
+   * @param logger logger
+   */
+  public static void loadAndPrintVersionInfo(Logger logger) {
+    Properties props = loadVersionProperties();
+    logger.info(props.getProperty(APP_BUILD_INFO));
+    logger.info("Compiled against Hadoop {}",
+                props.getProperty(HADOOP_BUILD_INFO));
+    logger.info(getHadoopVersionString());
+  }
+  
+  public static String getHadoopVersionString() {
+    return String.format(Locale.ENGLISH,
+        "Hadoop runtime version %s with source checksum %s and build date %s",
+        VersionInfo.getBranch(),
+        VersionInfo.getSrcChecksum(),
+        VersionInfo.getDate());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/AbstractInputPropertiesValidator.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/conf/AbstractInputPropertiesValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/AbstractInputPropertiesValidator.java
new file mode 100644
index 0000000..336b4dc
--- /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/conf/AbstractInputPropertiesValidator.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.conf;
+
+import org.apache.slider.core.exceptions.BadConfigException;
+
+/**
+ *
+ */
+public abstract class AbstractInputPropertiesValidator implements InputPropertiesValidator{
+
+  abstract void validatePropertyNamePrefix(String key) throws BadConfigException;
+
+  public void validate(ConfTreeOperations props)
+      throws BadConfigException {
+    validateGlobalProperties(props);
+    validateComponentProperties(props);
+
+  }
+
+  protected void validateComponentProperties(ConfTreeOperations props)
+      throws BadConfigException {
+    for (String compName : props.getComponentNames()) {
+      MapOperations mo = props.getComponent(compName);
+      if (mo == null) continue;
+      for (String key : mo.keySet()) {
+        validatePropertyNamePrefix(key);
+      }
+    }
+  }
+
+  abstract void validateGlobalProperties(ConfTreeOperations props)
+      throws BadConfigException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/AggregateConf.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/conf/AggregateConf.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/AggregateConf.java
new file mode 100644
index 0000000..18c3156
--- /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/conf/AggregateConf.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.conf;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.io.IOException;
+
+/**
+ * Aggregate Configuration.
+ *
+ * It is serializable to JSON
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public final class AggregateConf {
+
+  private String name;
+  private ConfTree resources;
+  private ConfTree internal;
+  private ConfTree appConf;
+
+  private ConfTreeOperations resourceOperations;
+  private ConfTreeOperations appConfOperations;
+  private ConfTreeOperations internalOperations;
+
+  private String passphrase;
+
+  public AggregateConf() {
+    this(new ConfTree(), new ConfTree(), new ConfTree());
+  }
+
+  public AggregateConf(String name) {
+    this(new ConfTree(), new ConfTree(), new ConfTree());
+    this.name = name;
+  }
+
+  public AggregateConf(ConfTree resources,
+                       ConfTree appConf,
+                       ConfTree internal) {
+    setResources(resources);
+    setAppConf(appConf);
+    setInternal(internal);
+  }
+
+  /**
+   * Take a snapshot of the configuration
+   * @param instanceDefinition source
+   * @throws IOException marshalling/copying problems
+   */
+  public AggregateConf(AggregateConf instanceDefinition) throws IOException {
+    ConfTreeOperations resourcesSnapshot =
+        ConfTreeOperations.fromInstance(instanceDefinition.getResources());
+    ConfTreeOperations appConfSnapshot =
+        ConfTreeOperations.fromInstance(instanceDefinition.getAppConf());
+    ConfTreeOperations internalsSnapshot =
+        ConfTreeOperations.fromInstance(instanceDefinition.getInternal());
+    //build a new aggregate from the snapshots
+    setResources(resourcesSnapshot.confTree);
+    setAppConf(appConfSnapshot.confTree);
+    setInternal(internalsSnapshot.confTree);
+  }
+  
+  public void setResources(ConfTree resources) {
+    this.resources = resources;
+    resourceOperations = new ConfTreeOperations(resources);
+  }
+
+  public void setAppConf(ConfTree appConf) {
+    this.appConf = appConf;
+    appConfOperations = new ConfTreeOperations(appConf);
+  }
+
+  public ConfTree getInternal() {
+    return internal;
+  }
+
+  public void setInternal(ConfTree internal) {
+    this.internal = internal;
+    internalOperations = new ConfTreeOperations(internal);
+  }
+
+  public ConfTree getResources() {
+    return resources;
+  }
+
+  public ConfTree getAppConf() {
+    return appConf;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  @JsonIgnore
+  public ConfTreeOperations getResourceOperations() {
+    return resourceOperations;
+  }
+
+
+  @JsonIgnore
+  public ConfTreeOperations getAppConfOperations() {
+    return appConfOperations;
+  }
+
+  @JsonIgnore
+  public ConfTreeOperations getInternalOperations() {
+    return internalOperations;
+  }
+
+  /**
+   * predicate to query if all sections have data structures
+   * @return true if every section is non-null
+   */
+  @JsonIgnore
+  public boolean isComplete() {
+    return resources != null && appConf != null && internal != null;
+  }
+
+  public void validate() throws BadConfigException {
+    if (!isComplete()) {
+      throw new BadConfigException("Incomplete instance %s", this);
+    }
+    resourceOperations.validate();
+    internalOperations.validate();
+    appConfOperations.validate();
+  }
+
+  public void resolve() throws BadConfigException {
+    validate();
+    resourceOperations.resolve();
+    internalOperations.resolve();
+    appConfOperations.resolve();
+  }
+
+  @JsonIgnore
+  public String getPassphrase() {
+    if (passphrase == null) {
+      passphrase = RandomStringUtils.randomAlphanumeric(
+          Integer.valueOf(SliderKeys.PASS_LEN));
+    }
+
+    return passphrase;
+  }
+
+  /**
+   * Is this app package versioned?
+   * 
+   * @return true if {@link SliderKeys#APP_VERSION} was set in the app config
+   *         provided during creation of this app
+   * @since 0.80.0-incubating
+   */
+  public boolean isVersioned() {
+    return StringUtils.isNotEmpty(getAppConfOperations().getGlobalOptions()
+        .get(SliderKeys.APP_VERSION));
+  }
+
+  /**
+   * string operation includes all the inner conftrees
+   * @return a string description
+   */
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+      new StringBuilder("{");
+    sb.append(",\n\"internal\": ").append(internal);
+    sb.append(",\n\"resources\": ").append(resources);
+    sb.append(",\n\"appConf\" :").append(appConf);
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTree.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/conf/ConfTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTree.java
new file mode 100644
index 0000000..be7c56f
--- /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/conf/ConfTree.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.conf;
+
+import org.apache.slider.core.persist.ConfTreeSerDeser;
+import org.apache.slider.core.persist.PersistKeys;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.JsonMappingException;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A conf tree represents one of the configuration trees
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public final class ConfTree {
+
+  /**
+   * Size of an initial map. This is kept low so the cost of having
+   * many conf trees in a process is low.
+   */
+  public static final int INITAL_MAP_CAPACITY = 3;
+
+  protected static final Logger
+    log = LoggerFactory.getLogger(ConfTree.class);
+
+  /**
+   * version counter
+   */
+  public String schema = PersistKeys.SCHEMA;
+
+  /**
+   * Metadata
+   */
+  public Map<String, Object> metadata = new HashMap<>(INITAL_MAP_CAPACITY);
+
+
+  /**
+   * Global options
+   */
+  public Map<String, String> global =
+    new HashMap<>(INITAL_MAP_CAPACITY);
+
+
+  /**
+   * Credentials
+   */
+  public Map<String, List<String>> credentials =
+      new HashMap<>(INITAL_MAP_CAPACITY);
+
+  /**
+   * Role options, 
+   * role -> option -> value
+   */
+  public Map<String, Map<String, String>> components =
+    new HashMap<>(INITAL_MAP_CAPACITY);
+
+
+  /**
+   * Shallow clone
+   * @return a shallow clone
+   * @throws CloneNotSupportedException
+   */
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return toJson();
+    } catch (Exception e) {
+      log.warn("Failed to convert to JSON ", e);
+      return super.toString();
+    }
+  }
+
+  /**
+   * Convert to a JSON string
+   * @return a JSON string description
+   * @throws IOException Problems mapping/writing the object
+   */
+  public String toJson() throws IOException,
+                                JsonGenerationException,
+                                JsonMappingException {
+    return ConfTreeSerDeser.toString(this);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.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/conf/ConfTreeOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
new file mode 100644
index 0000000..d24a158
--- /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/conf/ConfTreeOperations.java
@@ -0,0 +1,477 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.conf;
+
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.persist.ConfTreeSerDeser;
+import org.apache.slider.core.persist.PersistKeys;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.map.JsonMappingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class ConfTreeOperations {
+
+  public final ConfTree confTree;
+  private final MapOperations globalOptions;
+
+  protected static final Logger
+    log = LoggerFactory.getLogger(ConfTreeOperations.class);
+
+
+  public ConfTreeOperations(ConfTree confTree) {
+    assert confTree != null : "null tree";
+    assert confTree.components != null : "null tree components";
+    this.confTree = confTree;
+    globalOptions = new MapOperations("global", confTree.global);
+  }
+
+  /**
+   * Get the underlying conf tree
+   * @return the tree
+   */
+  public ConfTree getConfTree() {
+    return confTree;
+  }
+
+  /**
+   * Validate the configuration
+   * @throws BadConfigException
+   */
+  public void validate() throws BadConfigException {
+    validate(null);
+  }
+
+  /**
+   * Validate the configuration
+   * @param validator a provided properties validator
+   * @throws BadConfigException
+   */
+  public void validate(InputPropertiesValidator validator) throws BadConfigException {
+    String version = confTree.schema;
+    if (version == null) {
+      throw new BadConfigException("'version' undefined");
+    }
+    if (!PersistKeys.SCHEMA.equals(version)) {
+      throw new BadConfigException(
+          "version %s incompatible with supported version %s",
+          version,
+          PersistKeys.SCHEMA);
+    }
+    if (validator != null) {
+      validator.validate(this);
+    }
+  }
+
+  /**
+   * Resolve a ConfTree by mapping all global options into each component
+   * -if there is none there already
+   */
+  public void resolve() {
+    for (Map.Entry<String, Map<String, String>> comp : confTree.components.entrySet()) {
+      mergeInGlobal(comp.getValue());
+    }
+  }
+
+  /**
+   * Merge any options
+   * @param component dest values
+   */
+  public void mergeInGlobal(Map<String, String> component) {
+    SliderUtils.mergeMapsIgnoreDuplicateKeys(component, confTree.global);
+  }
+
+  /**
+   * Get operations on the global set
+   * @return a wrapped map
+   */
+  public MapOperations getGlobalOptions() {
+    return globalOptions;
+  }
+
+
+  /**
+   * look up a component and return its options
+   * @param component component name
+   * @return component mapping or null
+   */
+  public MapOperations getComponent(String component) {
+    Map<String, String> instance = confTree.components.get(component);
+    if (instance != null) {
+      return new MapOperations(component, instance);
+    }
+    return null;
+  }
+
+  /**
+   * look up a component and return its options with the specified replacements
+   * @param component component name
+   * @param replacementOptions replacement options
+   * @return component mapping or null
+   */
+  public MapOperations getComponent(String component, Map<String,String>
+      replacementOptions) {
+    Map<String, String> instance = confTree.components.get(component);
+    if (instance != null) {
+      Map<String, String> newInstance = new HashMap<>();
+      newInstance.putAll(instance);
+      newInstance.putAll(replacementOptions);
+      return new MapOperations(component, newInstance);
+    }
+    return null;
+  }
+
+  /**
+   * Get at the underlying component map
+   * @return a map of components. This is the raw ConfTree data structure
+   */
+  public Map<String, Map<String, String>> getComponents() {
+    return confTree.components;
+  }
+
+  /**
+   * Get a component -adding it to the components map if
+   * none with that name exists
+   * @param name role
+   * @return role mapping
+   */
+  public MapOperations getOrAddComponent(String name) {
+    MapOperations operations = getComponent(name);
+    if (operations != null) {
+      return operations;
+    }
+    //create a new instances
+    Map<String, String> map = new HashMap<>();
+    confTree.components.put(name, map);
+    return new MapOperations(name, map);
+  }
+
+
+  /*
+   * return the Set of names names
+   */
+  @JsonIgnore
+  public Set<String> getComponentNames() {
+    return new HashSet<String>(confTree.components.keySet());
+  }
+  
+  
+
+  /**
+   * Get a component whose presence is mandatory
+   * @param name component name
+   * @return the mapping
+   * @throws BadConfigException if the name is not there
+   */
+  public MapOperations getMandatoryComponent(String name) throws
+                                                          BadConfigException {
+    MapOperations ops = getComponent(name);
+    if (ops == null) {
+      throw new BadConfigException("Missing component " + name);
+    }
+    return ops;
+  }
+
+  /**
+   * Set a global option, converting it to a string as needed
+   * @param key key
+   * @param value non null value
+   */
+  public void set(String key, Object value) {
+    globalOptions.put(key, value.toString());
+  }
+  /**
+   * get a global option
+   * @param key key
+   * @return value or null
+   * 
+   */
+  public String get(String key) {
+    return globalOptions.get(key);
+  }
+
+  /**
+   * Propagate all global keys matching a prefix
+   * @param src source
+   * @param prefix prefix
+   */
+  public void propagateGlobalKeys(ConfTree src, String prefix) {
+    Map<String, String> global = src.global;
+    for (Map.Entry<String, String> entry : global.entrySet()) {
+      String key = entry.getKey();
+      if (key.startsWith(prefix)) {
+        set(key, entry.getValue());
+      }
+    }
+  }
+
+  /**
+   * Propagate all global keys matching a prefix
+   * @param src source
+   * @param prefix prefix
+   */
+  public void propagateGlobalKeys(ConfTreeOperations src, String prefix) {
+    propagateGlobalKeys(src.confTree, prefix);
+  }
+
+  /**
+   * Merge the map of a single component
+   * @param component component name
+   * @param map map to merge
+   */
+  public void mergeSingleComponentMap(String component, Map<String, String> map) {
+    MapOperations comp = getOrAddComponent(component);
+    comp.putAll(map);
+  }
+  /**
+   * Merge the map of a single component
+   * @param component component name
+   * @param map map to merge
+   */
+  public void mergeSingleComponentMapPrefix(String component,
+                                            Map<String, String> map,
+                                            String prefix,
+                                            boolean overwrite) {
+    MapOperations comp = getOrAddComponent(component);
+    comp.mergeMapPrefixedKeys(map,prefix, overwrite);
+  }
+
+  /**
+   * Merge in components
+   * @param commandOptions component options on the CLI
+   */
+  public void mergeComponents(Map<String, Map<String, String>> commandOptions) {
+    for (Map.Entry<String, Map<String, String>> entry : commandOptions.entrySet()) {
+      mergeSingleComponentMap(entry.getKey(), entry.getValue());
+    }
+  }
+
+  /**
+   * Merge in components
+   * @param commandOptions component options on the CLI
+   */
+  public void mergeComponentsPrefix(Map<String,
+    Map<String, String>> commandOptions,
+                                    String prefix,
+                                    boolean overwrite) {
+    for (Map.Entry<String, Map<String, String>> entry : commandOptions.entrySet()) {
+      mergeSingleComponentMapPrefix(entry.getKey(), entry.getValue(), prefix, overwrite);
+    }
+  }
+
+  /**
+   * Merge in another tree -no overwrites of global or conf data
+   * (note that metadata does a naive putAll merge/overwrite)
+   * @param that the other tree
+   */
+  public void mergeWithoutOverwrite(ConfTree that) {
+
+    getGlobalOptions().mergeWithoutOverwrite(that.global);
+    confTree.metadata.putAll(that.metadata);
+    confTree.credentials.putAll(that.credentials);
+
+    for (Map.Entry<String, Map<String, String>> entry : that.components.entrySet()) {
+      MapOperations comp = getOrAddComponent(entry.getKey());
+      comp.mergeWithoutOverwrite(entry.getValue());
+    }
+  }
+  
+  /**
+   * Merge in another tree with overwrites
+   * @param that the other tree
+   */
+  public void merge(ConfTree that) {
+
+    getGlobalOptions().putAll(that.global);
+    confTree.metadata.putAll(that.metadata);
+    confTree.credentials.putAll(that.credentials);
+
+    for (Map.Entry<String, Map<String, String>> entry : that.components.entrySet()) {
+      MapOperations comp = getOrAddComponent(entry.getKey());
+      comp.putAll(entry.getValue());
+    }
+  }
+
+  
+  /**
+   * Load from a resource. The inner conf tree is the loaded data -unresolved
+   * @param resource resource
+   * @return loaded value
+   * @throws IOException load failure
+   */
+  public static ConfTreeOperations fromResource(String resource) throws
+                                                                 IOException {
+    ConfTreeSerDeser confTreeSerDeser = new ConfTreeSerDeser();
+    ConfTreeOperations ops = new ConfTreeOperations(
+       confTreeSerDeser.fromResource(resource) );
+    return ops;      
+  }
+  
+  /**
+   * Load from a resource. The inner conf tree is the loaded data -unresolved
+   * @param resource resource
+   * @return loaded value
+   * @throws IOException load failure
+   */
+  public static ConfTreeOperations fromFile(File resource) throws
+                                                                 IOException {
+    ConfTreeSerDeser confTreeSerDeser = new ConfTreeSerDeser();
+    ConfTreeOperations ops = new ConfTreeOperations(
+       confTreeSerDeser.fromFile(resource) );
+    return ops;
+  }
+
+  /**
+   * Build from an existing instance -which is cloned via JSON ser/deser
+   * @param instance the source instance
+   * @return loaded value
+   * @throws IOException load failure
+   */
+  public static ConfTreeOperations fromInstance(ConfTree instance) throws
+                                                                 IOException {
+    ConfTreeSerDeser confTreeSerDeser = new ConfTreeSerDeser();
+    ConfTreeOperations ops = new ConfTreeOperations(
+       confTreeSerDeser.fromJson(confTreeSerDeser.toJson(instance)) );
+    return ops;
+  }
+
+  /**
+   * Load from a file and merge it in
+   * @param file file
+   * @throws IOException any IO problem
+   * @throws BadConfigException if the file is invalid
+   */
+  public void mergeFile(File file) throws IOException, BadConfigException {
+    mergeFile(file, null);
+  }
+
+  /**
+   * Load from a file and merge it in
+   * @param file file
+   * @param validator properties validator
+   * @throws IOException any IO problem
+   * @throws BadConfigException if the file is invalid
+   */
+  public void mergeFile(File file, InputPropertiesValidator validator) throws IOException, BadConfigException {
+    ConfTreeSerDeser confTreeSerDeser = new ConfTreeSerDeser();
+    ConfTree tree = confTreeSerDeser.fromFile(file);
+    ConfTreeOperations ops = new ConfTreeOperations(tree);
+    ops.validate(validator);
+    merge(ops.confTree);
+  }
+
+  @Override
+  public String toString() {
+    return confTree.toString();
+  }
+
+  /**
+   * Convert to a JSON string
+   * @return a JSON string description
+   */
+  public String toJson() throws IOException,
+                                JsonGenerationException,
+                                JsonMappingException {
+    return confTree.toJson();
+  }
+
+  /**
+   * Get a component option
+   * @param name component name
+   * @param option option name
+   * @param defVal default value
+   * @return resolved value
+   */
+  public String getComponentOpt(String name, String option, String defVal) {
+    MapOperations roleopts = getComponent(name);
+    if (roleopts == null) {
+      return defVal;
+    }
+    return roleopts.getOption(option, defVal);
+  }
+
+  /**
+   * Get a component opt; use {@link Integer#decode(String)} so as to take hex
+   * oct and bin values too.
+   *
+   * @param name component name
+   * @param option option name
+   * @param defVal default value
+   * @return parsed value
+   * @throws NumberFormatException if the role could not be parsed.
+   */
+  public int getComponentOptInt(String name, String option, int defVal) {
+    String val = getComponentOpt(name, option, Integer.toString(defVal));
+    return Integer.decode(val);
+  }
+
+  /**
+   * Get a component opt as a boolean using {@link Boolean#valueOf(String)}.
+   *
+   * @param name component name
+   * @param option option name
+   * @param defVal default value
+   * @return parsed value
+   * @throws NumberFormatException if the role could not be parsed.
+   */
+  public boolean getComponentOptBool(String name, String option, boolean defVal) {
+    String val = getComponentOpt(name, option, Boolean.toString(defVal));
+    return Boolean.valueOf(val);
+  }
+
+  /**
+   * Set a component option, creating the component if necessary
+   * @param component component name
+   * @param option option name
+   * @param val value
+   */
+  public void setComponentOpt(String component, String option, String val) {
+    Map<String, String> roleopts = getOrAddComponent(component);
+    roleopts.put(option, val);
+  }
+
+  /**
+   * Set an integer role option, creating the role if necessary
+   * @param role role name
+   * @param option option name
+   * @param val integer value
+   */
+  public void setComponentOpt(String role, String option, int val) {
+    setComponentOpt(role, option, Integer.toString(val));
+  }
+  /**
+   * Set a long role option, creating the role if necessary
+   * @param role role name
+   * @param option option name
+   * @param val long value
+   */
+  public void setComponentOpt(String role, String option, long val) {
+    setComponentOpt(role, option, Long.toString(val));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/InputPropertiesValidator.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/conf/InputPropertiesValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/InputPropertiesValidator.java
new file mode 100644
index 0000000..237c240
--- /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/conf/InputPropertiesValidator.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.conf;
+
+import org.apache.slider.core.exceptions.BadConfigException;
+
+/**
+ *
+ */
+public interface InputPropertiesValidator {
+  void validate(ConfTreeOperations props) throws BadConfigException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java
new file mode 100644
index 0000000..9714a0f
--- /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/conf/MapOperations.java
@@ -0,0 +1,344 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.conf;
+
+import com.google.common.base.Preconditions;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Standard map operations.
+ *
+ * This delegates the standard map interface to the map passed in,
+ * so it can be used to add more actions to the map.
+ */
+public class MapOperations implements Map<String, String> {
+  private static final Logger log =
+    LoggerFactory.getLogger(MapOperations.class);
+  public static final String DAYS = ".days";
+  public static final String HOURS = ".hours";
+  public static final String MINUTES = ".minutes";
+  public static final String SECONDS = ".seconds";
+
+  /**
+   * Global options
+   */
+  public final Map<String, String> options;
+
+  public final String name;
+
+  public MapOperations() {
+    options = new HashMap<String, String>();
+    name = "";
+  }
+
+  /**
+   * Create an instance
+   * @param name name
+   * @param options source of options
+   */
+  public MapOperations(String name, Map<String, String> options) {
+    Preconditions.checkArgument(options != null, "null map");
+    this.options = options;
+    this.name = name;
+  }
+
+  /**
+   * Create an instance from an iterative map entry
+   * @param entry entry to work with
+   */
+  public MapOperations(Map.Entry<String, Map<String, String>> entry) {
+    Preconditions.checkArgument(entry != null, "null entry");
+    this.name = entry.getKey();
+    this.options = entry.getValue();
+  }
+
+  /**
+   * Get an option value
+   *
+   * @param key key
+   * @param defVal default value
+   * @return option in map or the default
+   */
+  public String getOption(String key, String defVal) {
+    String val = options.get(key);
+    return val != null ? val : defVal;
+  }
+
+  /**
+   * Get a boolean option
+   *
+   * @param key option key
+   * @param defVal default value
+   * @return option true if the option equals "true", or the default value
+   * if the option was not defined at all.
+   */
+  public Boolean getOptionBool(String key, boolean defVal) {
+    String val = getOption(key, Boolean.toString(defVal));
+    return Boolean.valueOf(val);
+  }
+
+  /**
+   * Get a cluster option or value
+   *
+   * @param key option key
+   * @return the value
+   * @throws BadConfigException if the option is missing
+   */
+
+  public String getMandatoryOption(String key) throws BadConfigException {
+    String val = options.get(key);
+    if (val == null) {
+      if (log.isDebugEnabled()) {
+        log.debug("Missing key {} from config containing {}",
+                  key, this);
+      }
+      String text = "Missing option " + key;
+      if (SliderUtils.isSet(name)) {
+        text += " from set " + name;
+      }
+      throw new BadConfigException(text);
+    }
+    return val;
+  }
+
+  /**
+   * Get an integer option; use {@link Integer#decode(String)} so as to take hex
+   * oct and bin values too.
+   *
+   * @param option option name
+   * @param defVal default value
+   * @return parsed value
+   * @throws NumberFormatException if the role could not be parsed.
+   */
+  public int getOptionInt(String option, int defVal) {
+    String val = getOption(option, Integer.toString(defVal));
+    return Integer.decode(val);
+  }
+
+  /**
+   * Get a long option; use {@link Long#decode(String)} so as to take hex
+   * oct and bin values too.
+   *
+   * @param option option name
+   * @param defVal default value
+   * @return parsed value
+   * @throws NumberFormatException
+   */
+  public long getOptionLong(String option, long defVal) {
+    String val = getOption(option, Long.toString(defVal));
+    return Long.decode(val);
+  }
+
+  /**
+   * Get a mandatory integer option; use {@link Integer#decode(String)} so as to take hex
+   * oct and bin values too.
+   *
+   * @param option option name
+   * @return parsed value
+   * @throws NumberFormatException if the option could not be parsed.
+   * @throws BadConfigException if the option could not be found
+   */
+  public int getMandatoryOptionInt(String option) throws BadConfigException {
+    getMandatoryOption(option);
+    return getOptionInt(option, 0);
+  }
+
+  /**
+   * Verify that an option is set: that is defined AND non-empty
+   * @param key
+   * @throws BadConfigException
+   */
+  public void verifyOptionSet(String key) throws BadConfigException {
+    if (SliderUtils.isUnset(getOption(key, null))) {
+      throw new BadConfigException("Unset option %s", key);
+    }
+  }
+  
+  public void mergeWithoutOverwrite(Map<String, String> that) {
+    SliderUtils.mergeMapsIgnoreDuplicateKeys(options, that);
+  }
+
+  /**
+   * Merge a map by prefixed keys
+   * @param that the map to merge in
+   * @param prefix prefix to match on
+   * @param overwrite flag to enable overwrite
+   */
+  public void mergeMapPrefixedKeys(Map<String, String> that,
+                                    String prefix,
+                                    boolean overwrite) {
+    for (Map.Entry<String, String> entry : that.entrySet()) {
+      String key = entry.getKey();
+      if (key.startsWith(prefix)) {
+        if (overwrite || get(key) == null) {
+          put(key, entry.getValue());
+        }
+      }
+    }
+  }
+
+  /**
+   * Set a property if it is not already set
+   * @param key key
+   * @param value value
+   */
+  public void putIfUnset(String key, String value) {
+    if (get(key) == null) {
+      put(key, value);
+    }
+  }
+  
+  public void set(String key, Object value) {
+    assert value != null;
+    put(key, value.toString());
+  }
+
+  public int size() {
+    return options.size();
+  }
+
+  public boolean isEmpty() {
+    return options.isEmpty();
+  }
+
+  public boolean containsValue(Object value) {
+    return options.containsValue(value);
+  }
+
+  public boolean containsKey(Object key) {
+    return options.containsKey(key);
+  }
+
+  public String get(Object key) {
+    return options.get(key);
+  }
+
+  public String put(String key, String value) {
+    return options.put(key, value);
+  }
+
+  public String remove(Object key) {
+    return options.remove(key);
+  }
+
+  public void putAll(Map<? extends String, ? extends String> m) {
+    options.putAll(m);
+  }
+
+  public void clear() {
+    options.clear();
+  }
+
+  public Set<String> keySet() {
+    return options.keySet();
+  }
+
+  public Collection<String> values() {
+    return options.values();
+  }
+
+  public Set<Map.Entry<String, String>> entrySet() {
+    return options.entrySet();
+  }
+
+  @SuppressWarnings("EqualsWhichDoesntCheckParameterClass")
+  public boolean equals(Object o) {
+    return options.equals(o);
+  }
+
+  @Override
+  public int hashCode() {
+    return options.hashCode();
+  }
+
+  public boolean isSet(String key) {
+    return SliderUtils.isSet(get(key));
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append(name).append("=\n");
+
+    for (Entry<String, String> entry : options.entrySet()) {
+      builder.append("  ")
+             .append(entry.getKey())
+             .append('=')
+             .append(entry.getValue())
+             .append('\n');
+    }
+    return builder.toString();
+  }
+
+  /**
+   * Get the time range of a set of keys
+   * @param basekey base key to which suffix gets applied
+   * @param defDays
+   * @param defHours
+   * @param defMins
+   * @param defSecs
+   * @return the aggregate time range in seconds
+   */
+  public long getTimeRange(String basekey,
+      int defDays,
+      int defHours,
+      int defMins,
+      int defSecs) {
+    Preconditions.checkArgument(basekey != null);
+    int days = getOptionInt(basekey + DAYS, defDays);
+    int hours = getOptionInt(basekey + HOURS, defHours);
+
+    int minutes = getOptionInt(basekey + MINUTES, defMins);
+    int seconds = getOptionInt(basekey + SECONDS, defSecs);
+    // range check
+    Preconditions.checkState(days >= 0 && hours >= 0 && minutes >= 0
+                             && seconds >= 0,
+        "Time range for %s has negative time component %s:%s:%s:%s",
+        basekey, days, hours, minutes, seconds);
+
+    // calculate total time, schedule the reset if expected
+    long totalMinutes = (long) days * 24 * 60 + (long) hours * 24 + minutes;
+    return totalMinutes * 60 + seconds;
+  }
+
+  /**
+   * Get all entries with a specific prefix
+   * @param prefix prefix
+   * @return a prefixed map, possibly empty
+   */
+  public Map<String, String> prefixedWith(String prefix) {
+
+    Map<String, String> prefixed = new HashMap<>(size());
+    for (Entry<String, String> entry: entrySet()) {
+      if (entry.getKey().startsWith(prefix)) {
+        prefixed.put(entry.getKey(), entry.getValue());
+      }
+    }
+    return prefixed;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ResourcesInputPropertiesValidator.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/conf/ResourcesInputPropertiesValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ResourcesInputPropertiesValidator.java
new file mode 100644
index 0000000..19f6f8d
--- /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/conf/ResourcesInputPropertiesValidator.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.conf;
+
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.core.exceptions.BadConfigException;
+
+/**
+ *
+ */
+public class ResourcesInputPropertiesValidator
+    extends AbstractInputPropertiesValidator {
+
+  void validatePropertyNamePrefix(String key) throws BadConfigException {
+    if (!key.startsWith("yarn.") && !key.equals(ResourceKeys.UNIQUE_NAMES)) {
+      throw new BadConfigException(
+          "argument %s does not have 'yarn.' prefix", key);
+    }
+  }
+
+  protected void validateGlobalProperties(ConfTreeOperations props)
+      throws BadConfigException {
+    for (String key : props.getGlobalOptions().keySet()) {
+      validatePropertyNamePrefix(key);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/TemplateInputPropertiesValidator.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/conf/TemplateInputPropertiesValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/TemplateInputPropertiesValidator.java
new file mode 100644
index 0000000..aad2757
--- /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/conf/TemplateInputPropertiesValidator.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.conf;
+
+import org.apache.slider.core.exceptions.BadConfigException;
+
+/**
+ *
+ */
+public class TemplateInputPropertiesValidator
+    extends AbstractInputPropertiesValidator {
+
+  void validatePropertyNamePrefix(String key) throws BadConfigException {
+    if (key.startsWith("yarn.")) {
+      throw new BadConfigException(
+          "argument %s has 'yarn.' prefix - this is not allowed in templates", key);
+    }
+  }
+
+  @Override
+  void validateGlobalProperties(ConfTreeOperations props) {
+    // do nothing
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadClusterStateException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadClusterStateException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadClusterStateException.java
new file mode 100644
index 0000000..e73ce57
--- /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/exceptions/BadClusterStateException.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+
+/**
+ * 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/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadCommandArgumentsException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadCommandArgumentsException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadCommandArgumentsException.java
new file mode 100644
index 0000000..0d5d686
--- /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/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.slider.core.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/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadConfigException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadConfigException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadConfigException.java
new file mode 100644
index 0000000..65a8ea8
--- /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/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.slider.core.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/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java
new file mode 100644
index 0000000..8b04969
--- /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/exceptions/ErrorStrings.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+public interface ErrorStrings {
+  String E_UNSTABLE_CLUSTER = "Unstable Application Instance :";
+  String E_CLUSTER_RUNNING = "Application Instance running";
+  String E_ALREADY_EXISTS = "already exists";
+  String PRINTF_E_INSTANCE_ALREADY_EXISTS = "Application Instance \"%s\" already exists and is defined in %s";
+  String PRINTF_E_INSTANCE_DIR_ALREADY_EXISTS = "Application Instance dir already exists: %s";
+  String E_MISSING_PATH = "Missing path ";
+  String E_INCOMPLETE_CLUSTER_SPEC =
+    "Cluster specification is marked as incomplete: ";
+  String E_UNKNOWN_INSTANCE = "Unknown application instance ";
+  String E_DESTROY_CREATE_RACE_CONDITION =
+      "created while it was being destroyed";
+  String E_UNKNOWN_ROLE = "Unknown role ";
+  /**
+   * 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 ";
+  String E_APPLICATION_NOT_RUNNING = "Application not running";
+  String E_FINISHED_APPLICATION = E_APPLICATION_NOT_RUNNING + ": %s state=%s ";
+  String E_NO_IMAGE_OR_HOME_DIR_SPECIFIED =
+    "Neither an image path nor binary home directory were specified";
+  String E_BOTH_IMAGE_AND_HOME_DIR_SPECIFIED =
+    "Both application image path and home dir have been provided";
+  String E_CONFIGURATION_DIRECTORY_NOT_FOUND =
+    "Configuration directory \"%s\" not found";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java
new file mode 100644
index 0000000..efec676
--- /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/exceptions/ExceptionConverter.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.UniformInterfaceException;
+import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.fs.PathAccessDeniedException;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.yarn.webapp.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletResponse;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+/**
+ * static methods to convert exceptions into different types, including
+ * extraction of details and finer-grained conversions.
+ */
+public class ExceptionConverter {
+  private static final Logger
+      log = LoggerFactory.getLogger(ExceptionConverter.class);
+
+  /**
+   * Uprate error codes 400 and up into faults; 
+   * 404 is converted to a {@link FileNotFoundException},
+   * 401 to {@link ForbiddenException}
+   * FileNotFoundException for an unknown resource
+   * PathAccessDeniedException for access denied
+   * PathIOException for anything else
+   * @param verb HTTP Verb used
+   * @param targetURL URL being targeted 
+   * @param exception original exception
+   * @return a new exception, the original one nested as a cause
+   */
+  public static IOException convertJerseyException(String verb,
+      String targetURL,
+      UniformInterfaceException exception) {
+
+    IOException ioe = null;
+    ClientResponse response = exception.getResponse();
+    if (response != null) {
+      int status = response.getStatus();
+      String body = "";
+      try {
+        if (response.hasEntity()) {
+          body = response.getEntity(String.class);
+          log.error("{} {} returned status {} and body\n{}",
+              verb, targetURL, status, body);
+        } else {
+          log.error("{} {} returned status {} and empty body",
+              verb, targetURL, status);
+        }
+      } catch (Exception e) {
+        log.warn("Failed to extract body from client response", e);
+      }
+      
+      if (status == HttpServletResponse.SC_UNAUTHORIZED
+          || status == HttpServletResponse.SC_FORBIDDEN) {
+        ioe = new PathAccessDeniedException(targetURL);
+      } else if (status == HttpServletResponse.SC_BAD_REQUEST
+          || status == HttpServletResponse.SC_NOT_ACCEPTABLE
+          || status == HttpServletResponse.SC_UNSUPPORTED_MEDIA_TYPE) {
+        // bad request
+        ioe = new InvalidRequestException(
+            String.format("Bad %s request: status code %d against %s",
+                verb, status, targetURL));
+      } else if (status > 400 && status < 500) {
+        ioe =  new FileNotFoundException(targetURL);
+      }
+      if (ioe == null) {
+        ioe = new PathIOException(targetURL,
+            verb + " " + targetURL
+            + " failed with status code : " + status
+            + ":" + exception);
+      }
+    } else {
+      ioe = new PathIOException(targetURL, 
+          verb + " " + targetURL + " failed: " + exception);
+    }
+    ioe.initCause(exception);
+    return ioe; 
+  }
+
+  /**
+   * Handle a client-side Jersey exception.
+   * <p>
+   * If there's an inner IOException, return that.
+   * <p>
+   * Otherwise: create a new wrapper IOE including verb and target details
+   * @param verb HTTP Verb used
+   * @param targetURL URL being targeted 
+   * @param exception original exception
+   * @return an exception to throw
+   */
+  public static IOException convertJerseyException(String verb,
+      String targetURL,
+      ClientHandlerException exception) {
+    if (exception.getCause() instanceof IOException) {
+      return (IOException)exception.getCause();
+    } else {
+      IOException ioe = new IOException(
+          verb + " " + targetURL + " failed: " + exception);
+      ioe.initCause(exception);
+      return ioe;
+    } 
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NoSuchNodeException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NoSuchNodeException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NoSuchNodeException.java
new file mode 100644
index 0000000..ad2f1a4
--- /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/exceptions/NoSuchNodeException.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.exceptions;
+
+import java.io.IOException;
+
+/**
+ * Exception raised when a node cannot be found in the structure
+ * that is being examined.
+ */
+public class NoSuchNodeException extends IOException {
+
+  public NoSuchNodeException(String uuid) {
+    super(uuid);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NotFoundException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NotFoundException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NotFoundException.java
new file mode 100644
index 0000000..40cb94d
--- /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/exceptions/NotFoundException.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+
+/**
+ * Whatever was being resolved: it was not found
+ */
+public class NotFoundException extends SliderException {
+  public NotFoundException(String message,
+      Object... args) {
+    super(EXIT_NOT_FOUND, message, args);
+  }
+
+  public NotFoundException(Throwable throwable,
+      String message, Object... args) {
+    super(EXIT_NOT_FOUND, throwable, message, args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ServiceNotReadyException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ServiceNotReadyException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ServiceNotReadyException.java
new file mode 100644
index 0000000..435bc1a
--- /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/exceptions/ServiceNotReadyException.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+import java.io.IOException;
+
+/**
+ * This is an exception raised when the service does not consider itself
+ * live (yet)
+ */
+public class ServiceNotReadyException extends IOException {
+
+  public static final String E_NOT_READY =
+      "Service not ready for access: please retry";
+
+  public ServiceNotReadyException(String message) {
+    super(message);
+  }
+
+  public ServiceNotReadyException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public ServiceNotReadyException(Throwable cause) {
+    super(cause);
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/TriggerClusterTeardownException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/TriggerClusterTeardownException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/TriggerClusterTeardownException.java
new file mode 100644
index 0000000..bb9f430
--- /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/exceptions/TriggerClusterTeardownException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+
+/**
+ * An Exception to be thrown for an explicit "shut down the cluster" operation
+ * raised by the application state or other parts of the AM
+ */
+public class TriggerClusterTeardownException extends SliderException {
+
+  private final FinalApplicationStatus finalApplicationStatus;
+  
+  public TriggerClusterTeardownException(int code,
+      FinalApplicationStatus finalApplicationStatus, String message,
+      Object... args) {
+    super(code, message, args);
+    this.finalApplicationStatus = finalApplicationStatus;
+  }
+
+  public FinalApplicationStatus getFinalApplicationStatus() {
+    return finalApplicationStatus;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UnknownApplicationInstanceException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UnknownApplicationInstanceException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UnknownApplicationInstanceException.java
new file mode 100644
index 0000000..a1f8ae9
--- /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/exceptions/UnknownApplicationInstanceException.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+public class UnknownApplicationInstanceException extends SliderException {
+  public UnknownApplicationInstanceException(String s) {
+    super(EXIT_UNKNOWN_INSTANCE, s);
+  }
+
+  public UnknownApplicationInstanceException(String s, Throwable throwable) {
+    super(EXIT_UNKNOWN_INSTANCE, throwable, s);
+  }
+
+  public UnknownApplicationInstanceException(String message,
+      Object... args) {
+    super(EXIT_UNKNOWN_INSTANCE, message, args);
+  }
+
+  /**
+   * Create an instance with the standard exception name
+   * @param name name
+   * @return an instance to throw
+   */
+  public static UnknownApplicationInstanceException unknownInstance(String name) {
+    return new UnknownApplicationInstanceException(ErrorStrings.E_UNKNOWN_INSTANCE
+                                   + ": " + name);
+  }
+  public static UnknownApplicationInstanceException unknownInstance(String name,
+      Throwable throwable) {
+    UnknownApplicationInstanceException exception =
+      unknownInstance(name);
+    exception.initCause(throwable);
+    return exception;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UsageException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UsageException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UsageException.java
new file mode 100644
index 0000000..8684294
--- /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/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.slider.core.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/31c4a419/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/WaitTimeoutException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/WaitTimeoutException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/WaitTimeoutException.java
new file mode 100644
index 0000000..5ad3fdc
--- /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/exceptions/WaitTimeoutException.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.slider.core.exceptions;
+
+import java.io.IOException;
+
+/**
+ * Called when some spinning operation timed out
+ */
+public class WaitTimeoutException extends IOException {
+  public WaitTimeoutException(String message) {
+    super(message);
+  }
+
+  public WaitTimeoutException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}


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