You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by sm...@apache.org on 2015/03/18 05:10:28 UTC

[3/5] incubator-slider git commit: SLIDER-663. Make it easy to develop and deploy application packages that are essentially shell commands (part-I)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
index 418868c..3f1b7fe 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
@@ -19,6 +19,7 @@ package org.apache.slider.providers.agent.application.metadata;
 import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.exceptions.BadConfigException;
 import org.apache.slider.core.exceptions.SliderException;
+import org.codehaus.jackson.annotate.JsonProperty;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -26,21 +27,28 @@ import java.util.List;
 /**
  *
  */
-public class Component {
+public class Component implements Validate {
+
+  public static String TYPE_STANDARD = "STANDARD";
+  public static String TYPE_DOCKER = "DOCKER";
+  public static String CATEGORY_MASTER = "MASTER";
+  public static String CATEGORY_SLAVE = "SLAVE";
+  public static String CATEGORY_CLIENT = "CLIENT";
+
   String name;
-  String category;
-  String publishConfig;
-  String minInstanceCount;
+  String category = CATEGORY_MASTER;
+  String publishConfig = Boolean.FALSE.toString();
+  String minInstanceCount = "0";
   String maxInstanceCount;
-  String autoStartOnFailure;
+  String autoStartOnFailure = Boolean.FALSE.toString();
   String appExports;
   String compExports;
   CommandScript commandScript;
-  List<ComponentExport> componentExports;
+  String type = TYPE_STANDARD;
+  List<ComponentExport> componentExports = new ArrayList<>();
+  List<ComponentCommand> commands = new ArrayList<>();
 
   public Component() {
-    publishConfig = Boolean.FALSE.toString();
-    componentExports = new ArrayList<ComponentExport>();
   }
 
   public String getName() {
@@ -51,6 +59,14 @@ public class Component {
     this.name = name;
   }
 
+  public String getType() {
+    return type;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
   public String getCategory() {
     return category;
   }
@@ -95,6 +111,14 @@ public class Component {
     return minInstanceCount;
   }
 
+  public Boolean getAutoStartOnFailureBoolean() {
+    if (SliderUtils.isUnset(getAutoStartOnFailure())) {
+      return Boolean.FALSE;
+    }
+
+    return Boolean.parseBoolean(getAutoStartOnFailure());
+  }
+
   public int getMinInstanceCountInt() throws BadConfigException {
     if (SliderUtils.isUnset(minInstanceCount)) {
       return 0;
@@ -151,6 +175,11 @@ public class Component {
     return Boolean.parseBoolean(this.autoStartOnFailure);
   }
 
+  @JsonProperty("commands")
+  public List<ComponentCommand> getCommands() {
+    return this.commands;
+  }
+
   @Override
   public String toString() {
     final StringBuilder sb =
@@ -162,9 +191,39 @@ public class Component {
     return sb.toString();
   }
 
-  class AutoRestartSettings {
-    private boolean requiresAutoRestart;
-    private int maxFailures;
-    private int inThisManyMinutes;
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getName(), "name", "component");
+    Metainfo.checkNonNull(getCategory(), "category", "component");
+    if (!getCategory().equals(CATEGORY_MASTER)
+        && !getCategory().equals(CATEGORY_SLAVE)
+        && !getCategory().equals(CATEGORY_CLIENT)) {
+      throw new SliderException("Invalid category for the component " + getCategory());
+    }
+
+    Metainfo.checkNonNull(getType(), "type", "component");
+    if (!getType().equals(TYPE_DOCKER)
+        && !getType().equals(TYPE_STANDARD)) {
+      throw new SliderException("Invalid type for the component " + getType());
+    }
+
+    if (version.equals(Metainfo.VERSION_TWO_ZERO)) {
+      if (getType().equals(TYPE_DOCKER)) {
+        throw new SliderException(TYPE_DOCKER + " is not supported in version " + Metainfo.VERSION_TWO_ZERO);
+      }
+
+      if (getCommands().size() > 0) {
+        throw new SliderException("commands are not supported in version " + Metainfo.VERSION_TWO_ZERO);
+      }
+    }
+
+    if (commandScript != null) {
+      commandScript.validate(version);
+    }
+
+    if (version.equals(Metainfo.VERSION_TWO_ONE)) {
+      for (ComponentCommand cc : getCommands()) {
+        cc.validate(version);
+      }
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentCommand.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentCommand.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentCommand.java
new file mode 100644
index 0000000..52117c5
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentCommand.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Represents the metadata associated with the application.
+ */
+public class ComponentCommand implements Validate {
+  protected static final Logger
+      log = LoggerFactory.getLogger(ComponentCommand.class);
+
+
+  private String exec;
+  private String name = "START";
+  private String type = "SHELL";
+
+  /**
+   * Creator.
+   */
+  public ComponentCommand() {
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setExec(String exec) {
+    this.exec = exec;
+  }
+
+  public String getExec() {
+    return exec;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getName(), "name", "componentCommand");
+
+    Metainfo.checkNonNull(getType(), "version", "application");
+  }
+
+  public static ComponentCommand getDefaultComponentCommand() {
+    ComponentCommand cc = new ComponentCommand();
+    cc.setExec("DEFAULT");
+    return cc;
+  }
+
+  public static ComponentCommand getDefaultComponentCommand(String commandName) {
+    ComponentCommand cc = new ComponentCommand();
+    cc.setExec("DEFAULT");
+    cc.setName(commandName);
+    return cc;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ConfigFile.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ConfigFile.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ConfigFile.java
index b9dfb4e..cb47512 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ConfigFile.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ConfigFile.java
@@ -16,10 +16,12 @@
  */
 package org.apache.slider.providers.agent.application.metadata;
 
+import org.apache.slider.core.exceptions.SliderException;
+
 /**
  *
  */
-public class ConfigFile {
+public class ConfigFile implements Validate {
   String type;
   String fileName;
   String dictionaryName;
@@ -50,4 +52,8 @@ public class ConfigFile {
   public void setDictionaryName(String dictionaryName) {
     this.dictionaryName = dictionaryName;
   }
+
+  public void validate(String version) throws SliderException {
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Export.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Export.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Export.java
index 17326a3..5e0fb24 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Export.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Export.java
@@ -16,10 +16,12 @@
  */
 package org.apache.slider.providers.agent.application.metadata;
 
+import org.apache.slider.core.exceptions.SliderException;
+
 /**
  *
  */
-public class Export {
+public class Export implements Validate {
   String name;
   String value;
 
@@ -51,4 +53,9 @@ public class Export {
     sb.append('}');
     return sb.toString();
   }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getName(), "name", "export");
+    Metainfo.checkNonNull(getValue(), "value", "export");
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ExportGroup.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ExportGroup.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ExportGroup.java
index d2e20a4..3d9f34c 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ExportGroup.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ExportGroup.java
@@ -16,13 +16,15 @@
  */
 package org.apache.slider.providers.agent.application.metadata;
 
+import org.apache.slider.core.exceptions.SliderException;
+
 import java.util.ArrayList;
 import java.util.List;
 
 /**
  *
  */
-public class ExportGroup {
+public class ExportGroup implements Validate {
   String name;
   List<Export> exports;
 
@@ -59,4 +61,11 @@ public class ExportGroup {
     sb.append('}');
     return sb.toString();
   }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getName(), "name", "exportGroup");
+    for(Export exp : getExports()) {
+      exp.validate(version);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
index b34cba1..f89846e 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
@@ -16,6 +16,8 @@
  */
 package org.apache.slider.providers.agent.application.metadata;
 
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.SliderException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -25,13 +27,12 @@ import org.slf4j.LoggerFactory;
 public class Metainfo {
   protected static final Logger log =
       LoggerFactory.getLogger(Metainfo.class);
+  public static String VERSION_TWO_ZERO = "2.0";
+  public static String VERSION_TWO_ONE = "2.1";
 
   String schemaVersion;
   Application application;
 
-  public Metainfo() {
-  }
-
   public String getSchemaVersion() {
     return schemaVersion;
   }
@@ -50,7 +51,7 @@ public class Metainfo {
 
   public Component getApplicationComponent(String roleName) {
     if (application == null) {
-      log.error("Malformed app definition: Expect application as the top level element for metainfo.xml");
+      log.error("Malformed app definition: Expect application as the top level element for metainfo");
     } else {
       for (Component component : application.getComponents()) {
         if (component.getName().equals(roleName)) {
@@ -60,4 +61,19 @@ public class Metainfo {
     }
     return null;
   }
+
+  public void validate() throws SliderException {
+    if (!VERSION_TWO_ONE.equals(schemaVersion) ||
+        !VERSION_TWO_ZERO.equals(schemaVersion)) {
+      throw new SliderException("Unsupported version " + getSchemaVersion());
+    }
+
+    application.validate(schemaVersion);
+  }
+
+  public static void checkNonNull(String value, String field, String type) throws SliderException {
+    if (SliderUtils.isUnset(value)) {
+      throw new SliderException(type + "." + field + " cannot be null");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
index 1d8403f..12af586 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
@@ -16,18 +16,78 @@
  */
 package org.apache.slider.providers.agent.application.metadata;
 
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
 import org.apache.commons.digester.Digester;
+import org.apache.commons.io.IOUtils;
 import org.xml.sax.SAXException;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.StringWriter;
 
 /**
  *
  */
 public class MetainfoParser {
+  private final GsonBuilder gsonBuilder = new GsonBuilder();
+  private final Gson gson;
 
-  public Metainfo parse(InputStream metainfoStream) throws IOException {
+  public MetainfoParser() {
+    gson = gsonBuilder.create();
+  }
+
+  /**
+   * Convert to a JSON string
+   *
+   * @return a JSON string description
+   *
+   * @throws IOException Problems mapping/writing the object
+   */
+  public String toJsonString(Metainfo metaInfo) throws IOException {
+    return gson.toJson(metaInfo);
+  }
+
+  /**
+   * Convert from JSON
+   *
+   * @param json input
+   *
+   * @return the parsed JSON
+   *
+   * @throws IOException IO
+   */
+  public Metainfo fromJsonString(String json)
+      throws IOException {
+    return gson.fromJson(json, Metainfo.class);
+  }
+
+  /**
+   * Parse metainfo from an IOStream
+   *
+   * @param is
+   *
+   * @return
+   *
+   * @throws IOException
+   */
+  public Metainfo fromJsonStream(InputStream is) throws IOException {
+    StringWriter writer = new StringWriter();
+    IOUtils.copy(is, writer);
+    return fromJsonString(writer.toString());
+  }
+
+
+  /**
+   * Parse metainfo from an XML formatted IOStream
+   *
+   * @param metainfoStream
+   *
+   * @return
+   *
+   * @throws IOException
+   */
+  public Metainfo fromXmlStream(InputStream metainfoStream) throws IOException {
     Digester digester = new Digester();
     digester.setValidating(false);
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSPackage.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSPackage.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSPackage.java
index 334f96b..32b4890 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSPackage.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSPackage.java
@@ -16,10 +16,12 @@
  */
 package org.apache.slider.providers.agent.application.metadata;
 
+import org.apache.slider.core.exceptions.SliderException;
+
 /**
  *
  */
-public class OSPackage {
+public class OSPackage implements Validate {
   String type;
   String name;
 
@@ -41,4 +43,9 @@ public class OSPackage {
   public void setName(String name) {
     this.name = name;
   }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getName(), "name", "osPackage");
+    Metainfo.checkNonNull(getType(), "type", "osPackage");
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSSpecific.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSSpecific.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSSpecific.java
index 7c36e8e..c06d498 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSSpecific.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSSpecific.java
@@ -16,13 +16,15 @@
  */
 package org.apache.slider.providers.agent.application.metadata;
 
+import org.apache.slider.core.exceptions.SliderException;
+
 import java.util.ArrayList;
 import java.util.List;
 
 /**
  *
  */
-public class OSSpecific {
+public class OSSpecific implements Validate {
   String osType;
   List<OSPackage> packages;
 
@@ -45,4 +47,11 @@ public class OSSpecific {
   public List<OSPackage> getPackages() {
     return packages;
   }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getOsType(), "osType", "osSpecific");
+    for (OSPackage opkg : getPackages()) {
+      opkg.validate(version);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Package.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Package.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Package.java
new file mode 100644
index 0000000..b88f77d
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Package.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Represents package description.
+ */
+public class Package implements Validate {
+  protected static final Logger
+      log = LoggerFactory.getLogger(Package.class);
+
+
+  private String name;
+  private String type;
+
+  public Package() {
+  }
+
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getName(), "name", "package");
+    Metainfo.checkNonNull(getType(), "type", "package");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Validate.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Validate.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Validate.java
new file mode 100644
index 0000000..ef03dcd
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Validate.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.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+
+/**
+ * Implementer provides a validate method
+ */
+public interface Validate {
+
+  public void validate(String version) throws SliderException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ExecutionCommand.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ExecutionCommand.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ExecutionCommand.java
index 5fb3b5e..eca5e7d 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ExecutionCommand.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ExecutionCommand.java
@@ -16,11 +16,11 @@
  */
 package org.apache.slider.server.appmaster.web.rest.agent;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.annotate.JsonProperty;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -31,7 +31,8 @@ import java.util.Map;
 @JsonIgnoreProperties(ignoreUnknown = true)
 @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 public class ExecutionCommand {
-  private static Log LOG = LogFactory.getLog(ExecutionCommand.class);
+  protected static final Logger log =
+      LoggerFactory.getLogger(ExecutionCommand.class);
   private AgentCommandType commandType = AgentCommandType.EXECUTION_COMMAND;
   private String clusterName;
   private long taskId;
@@ -46,6 +47,7 @@ public class ExecutionCommand {
   private Map<String, String> commandParams;
   private String serviceName;
   private String componentName;
+  private String componentType;
 
   public ExecutionCommand(AgentCommandType commandType) {
     this.commandType = commandType;
@@ -121,6 +123,16 @@ public class ExecutionCommand {
     this.clusterName = clusterName;
   }
 
+  @JsonProperty("componentType")
+  public String getComponentType() {
+    return componentType;
+  }
+
+  @JsonProperty("componentType")
+  public void setComponentType(String componentType) {
+    this.componentType = componentType;
+  }
+
   @JsonProperty("hostname")
   public String getHostname() {
     return hostname;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/appConfig.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/appConfig.json b/slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/appConfig.json
new file mode 100644
index 0000000..051a247
--- /dev/null
+++ b/slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/appConfig.json
@@ -0,0 +1,8 @@
+{
+  "schema": "http://example.org/specification/v2.0.0",
+  "metadata": {
+  },
+  "global": {
+    "site.global.port": "${LISTEN.ALLOCATED_PORT}{PER_CONTAINER}"
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/metainfo.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/metainfo.json b/slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/metainfo.json
new file mode 100644
index 0000000..3c7391c
--- /dev/null
+++ b/slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/metainfo.json
@@ -0,0 +1,28 @@
+{
+  "schemaVersion": "2.1",
+  "application": {
+    "name": "LISTENER",
+    "exportGroups": [
+      {
+        "name": "Servers",
+        "exports": [
+          {
+            "name": "host_port",
+            "value": "${LISTEN_HOST}:${site.global.port}"
+          }
+        ]
+      }
+    ],
+    "components": [
+      {
+        "name": "LISTEN",
+        "compExports": "Servers-host_port",
+        "commands": [
+          {
+            "exec": "nc -l {$conf:@//site/global/port}"
+          }
+        ]
+      }
+    ]
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/resources.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/resources.json b/slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/resources.json
new file mode 100644
index 0000000..b3ee0cf
--- /dev/null
+++ b/slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/resources.json
@@ -0,0 +1,17 @@
+{
+  "schema" : "http://example.org/specification/v2.0.0",
+  "metadata" : {
+  },
+  "global" : {
+  },
+  "components": {
+    "slider-appmaster": {
+      "yarn.memory": "256"
+    },
+    "LISTEN": {
+      "yarn.role.priority": "1",
+      "yarn.component.instances": "1",
+      "yarn.memory": "256"
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/test/app_packages/test_min_pkg/sleep_cmd/metainfo.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_min_pkg/sleep_cmd/metainfo.json b/slider-core/src/test/app_packages/test_min_pkg/sleep_cmd/metainfo.json
new file mode 100644
index 0000000..c8ada2c
--- /dev/null
+++ b/slider-core/src/test/app_packages/test_min_pkg/sleep_cmd/metainfo.json
@@ -0,0 +1,16 @@
+{
+    "schemaVersion": "2.1",
+    "application": {
+        "name": "SLEEPER",
+        "components": [
+            {
+                "name": "SLEEP_100",
+                "commands": [
+                    {
+                        "exec": "sleep 180"
+                    }
+                ]
+             }
+        ]
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/test/app_packages/test_min_pkg/sleep_cmd/resources.json
----------------------------------------------------------------------
diff --git a/slider-core/src/test/app_packages/test_min_pkg/sleep_cmd/resources.json b/slider-core/src/test/app_packages/test_min_pkg/sleep_cmd/resources.json
new file mode 100644
index 0000000..be7b962
--- /dev/null
+++ b/slider-core/src/test/app_packages/test_min_pkg/sleep_cmd/resources.json
@@ -0,0 +1,17 @@
+{
+  "schema" : "http://example.org/specification/v2.0.0",
+  "metadata" : {
+  },
+  "global" : {
+  },
+  "components": {
+    "slider-appmaster": {
+      "yarn.memory": "256"
+    },
+    "SLEEP_100": {
+      "yarn.role.priority": "1",
+      "yarn.component.instances": "1",
+      "yarn.memory": "256"
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider2.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider2.java b/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider2.java
index 3bc7ce2..4bd9842 100644
--- a/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider2.java
+++ b/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider2.java
@@ -33,8 +33,7 @@ import org.apache.slider.core.exceptions.SliderException;
 import org.apache.slider.providers.ProviderUtils;
 import org.apache.slider.providers.agent.application.metadata.Application;
 import org.apache.slider.providers.agent.application.metadata.Metainfo;
-import org.apache.slider.providers.agent.application.metadata.OSPackage;
-import org.apache.slider.providers.agent.application.metadata.OSSpecific;
+import org.apache.slider.providers.agent.application.metadata.Package;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -126,13 +125,10 @@ public class TestAgentClientProvider2 {
     Metainfo metainfo = new Metainfo();
     Application app = new Application();
     metainfo.setApplication(app);
-    OSSpecific osSpecific = new OSSpecific();
-    osSpecific.setOsType("any");
-    app.addOSSpecific(osSpecific);
-    OSPackage pkg = new OSPackage();
-    osSpecific.addOSPackage(pkg);
+    Package pkg = new Package();
     pkg.setName("app.tar");
     pkg.setType("tarball");
+    app.getPackages().add(pkg);
 
     File clientInstallPath = new File("/tmp/file1");
     String appName = "name";
@@ -267,7 +263,7 @@ public class TestAgentClientProvider2 {
       client.actionClient(args);
     }catch(SliderException e) {
       log.info(e.getMessage());
-      Assert.assertTrue(e.getMessage().contains("Not a valid app package. Could not read metainfo.xml"));
+      Assert.assertTrue(e.getMessage().contains("Not a valid app package. Could not read metainfo"));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java b/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
index 38e7db1..f1d2c10 100644
--- a/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
+++ b/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
@@ -52,6 +52,7 @@ import org.apache.slider.providers.agent.application.metadata.Application;
 import org.apache.slider.providers.agent.application.metadata.CommandOrder;
 import org.apache.slider.providers.agent.application.metadata.CommandScript;
 import org.apache.slider.providers.agent.application.metadata.Component;
+import org.apache.slider.providers.agent.application.metadata.ComponentCommand;
 import org.apache.slider.providers.agent.application.metadata.ComponentExport;
 import org.apache.slider.providers.agent.application.metadata.ConfigFile;
 import org.apache.slider.providers.agent.application.metadata.DefaultConfig;
@@ -264,7 +265,7 @@ public class TestAgentProviderService {
                                                + "</metainfo>";
 
   @Test
-  public void testRegistration() throws IOException {
+  public void testRegistration() throws Exception {
 
     ConfTree tree = new ConfTree();
     tree.global.put(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH, ".");
@@ -305,6 +306,11 @@ public class TestAgentProviderService {
     doReturn(cs).when(mockAps).getScriptPathFromMetainfo(anyString());
     Metainfo metainfo = new Metainfo();
     metainfo.setApplication(new Application());
+
+    Component hm = new Component();
+    hm.setName("HBASE_MASTER");
+    metainfo.getApplication().addComponent(hm);
+
     doReturn(metainfo).when(mockAps).getApplicationMetainfo(any(SliderFileSystem.class), anyString());
 
     Configuration conf = new Configuration();
@@ -318,6 +324,7 @@ public class TestAgentProviderService {
           eq("mockcontainer_1"),
           any(HeartBeatResponse.class),
           eq("scripts/hbase_master.py"),
+          eq((ComponentCommand)null),
           eq(600L));
       doReturn(conf).when(mockAps).getConfig();
     } catch (SliderException e) {
@@ -422,7 +429,7 @@ public class TestAgentProviderService {
   }
 
   private AgentProviderService prepareProviderServiceForAgentStateTests()
-      throws IOException {
+      throws Exception {
     ContainerLaunchContext ctx = createNiceMock(ContainerLaunchContext.class);
     Container container = createNiceMock(Container.class);
     String role = "HBASE_MASTER";
@@ -455,7 +462,7 @@ public class TestAgentProviderService {
     metainfo.setApplication(application);
     doReturn(metainfo).when(mockAps).getApplicationMetainfo(
         any(SliderFileSystem.class), anyString());
-    doReturn(metainfo).when(mockAps).getMetainfo();
+    doReturn(metainfo).when(mockAps).getMetaInfo();
 
 
 
@@ -463,13 +470,14 @@ public class TestAgentProviderService {
       doReturn(true).when(mockAps).isMaster(anyString());
       doNothing().when(mockAps).addInstallCommand(eq("HBASE_MASTER"),
           eq("mockcontainer_1"), any(HeartBeatResponse.class),
-          eq("scripts/hbase_master.py"), eq(600L));
+          eq("scripts/hbase_master.py"), eq((ComponentCommand)null),
+          eq(600L));
       doReturn(conf).when(mockAps).getConfig();
     } catch (SliderException e) {
     }
 
     doNothing().when(mockAps).processAllocatedPorts(anyString(), anyString(),
-        anyString(), anyMap());
+                                                    anyString(), anyMap());
     expect(access.isApplicationLive()).andReturn(true).anyTimes();
     ClusterDescription desc = new ClusterDescription();
     desc.setOption(OptionKeys.ZOOKEEPER_QUORUM, "host1:2181");
@@ -495,14 +503,14 @@ public class TestAgentProviderService {
     ProviderRole providerRole = new ProviderRole(role, 1);
     providerRoleMap.put(1, providerRole);
     mockAps.rebuildContainerDetails(containers, "mockcontainer_1",
-        providerRoleMap);
+                                    providerRoleMap);
     return mockAps;
   }
 
   @Test
   public void testThreeInstallFailures() throws IOException, SliderException {
     InputStream metainfo_1 = new ByteArrayInputStream(metainfo_1_str.getBytes());
-    Metainfo metainfo = new MetainfoParser().parse(metainfo_1);
+    Metainfo metainfo = new MetainfoParser().fromXmlStream(metainfo_1);
     ConfTree tree = new ConfTree();
     tree.global.put(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH, ".");
 
@@ -553,6 +561,7 @@ public class TestAgentProviderService {
           anyString(),
           any(HeartBeatResponse.class),
           anyString(),
+          eq((ComponentCommand)null),
           Mockito.anyLong());
       doReturn(conf).when(mockAps).getConfig();
     } catch (SliderException e) {
@@ -621,6 +630,7 @@ public class TestAgentProviderService {
                                                                   anyString(),
                                                                   any(HeartBeatResponse.class),
                                                                   anyString(),
+                                                                  eq((ComponentCommand)null),
                                                                   Mockito.anyLong());
     } catch (SliderException he) {
       log.warn(he.getMessage());
@@ -630,14 +640,23 @@ public class TestAgentProviderService {
   }
 
   @Test
-  public void testAgentStateStarted() throws IOException, SliderException {
+  public void testAgentStateStarted() throws Exception {
     AggregateConf instanceDefinition = prepareConfForAgentStateTests();
     AgentProviderService mockAps = prepareProviderServiceForAgentStateTests();
+
+    Metainfo metainfo = new Metainfo();
+    Application application = new Application();
+    Component hbaseMaster = new Component();
+    hbaseMaster.setName("HBASE_MASTER");
+    application.addComponent(hbaseMaster);
+    metainfo.setApplication(application);
+    doReturn(metainfo).when(mockAps).getMetaInfo();
+
     Register reg = new Register();
     reg.setResponseId(0);
     reg.setLabel("mockcontainer_1___HBASE_MASTER");
     Map<String,String> ports = new HashMap<String,String>();
-    ports.put("a","100");
+    ports.put("a", "100");
     reg.setAllocatedPorts(ports);
 
     // Simulating agent in STARTED state
@@ -683,7 +702,7 @@ public class TestAgentProviderService {
   }
 
   @Test
-  public void testAgentStateInstalled() throws IOException, SliderException {
+  public void testAgentStateInstalled() throws Exception, SliderException {
     AggregateConf instanceDefinition = prepareConfForAgentStateTests();
     AgentProviderService mockAps = prepareProviderServiceForAgentStateTests();
 
@@ -692,11 +711,14 @@ public class TestAgentProviderService {
     CommandOrder cmdOrder = new CommandOrder();
     cmdOrder.setCommand("HBASE_MASTER-START");
     cmdOrder.setRequires("HBASE_MASTER-INSTALLED");
-    application.addCommandOrder(cmdOrder);
+    application.getCommandOrders().add(cmdOrder);
+    Component hbaseMaster = new Component();
+    hbaseMaster.setName("HBASE_MASTER");
+    application.addComponent(hbaseMaster);
     metainfo.setApplication(application);
     doReturn(metainfo).when(mockAps).getApplicationMetainfo(
         any(SliderFileSystem.class), anyString());
-    doReturn(metainfo).when(mockAps).getMetainfo();
+    doReturn(metainfo).when(mockAps).getMetaInfo();
     doNothing().when(mockAps).addRoleRelatedTokens(anyMap());
 
     Register reg = new Register();
@@ -810,11 +832,11 @@ public class TestAgentProviderService {
   @Test
   public void testComponentSpecificPublishes() throws Exception {
     InputStream metainfo_1 = new ByteArrayInputStream(metainfo_1_str.getBytes());
-    Metainfo metainfo = new MetainfoParser().parse(metainfo_1);
+    Metainfo metainfo = new MetainfoParser().fromXmlStream(metainfo_1);
     AgentProviderService aps = createAgentProviderService(new Configuration());
     AgentProviderService mockAps = Mockito.spy(aps);
     doNothing().when(mockAps).publishApplicationInstanceData(anyString(), anyString(), anyCollection());
-    doReturn(metainfo).when(mockAps).getMetainfo();
+    doReturn(metainfo).when(mockAps).getMetaInfo();
 
     Map<String, String> ports = new HashMap<String, String>();
     ports.put("global.listen_port", "10010");
@@ -846,15 +868,14 @@ public class TestAgentProviderService {
     }
   }
 
-
   @Test
   public void testComponentSpecificPublishes2() throws Exception {
     InputStream metainfo_1 = new ByteArrayInputStream(metainfo_1_str.getBytes());
-    Metainfo metainfo = new MetainfoParser().parse(metainfo_1);
+    Metainfo metainfo = new MetainfoParser().fromXmlStream(metainfo_1);
     AgentProviderService aps = createAgentProviderService(new Configuration());
     AgentProviderService mockAps = Mockito.spy(aps);
     doNothing().when(mockAps).publishApplicationInstanceData(anyString(), anyString(), anyCollection());
-    doReturn(metainfo).when(mockAps).getMetainfo();
+    doReturn(metainfo).when(mockAps).getMetaInfo();
     StateAccessForProviders access = createNiceMock(StateAccessForProviders.class);
     doReturn(access).when(mockAps).getAmState();
     PublishedExportsSet pubExpSet = new PublishedExportsSet();
@@ -935,7 +956,7 @@ public class TestAgentProviderService {
   @Test
   public void testProcessConfig() throws Exception {
     InputStream metainfo_1 = new ByteArrayInputStream(metainfo_1_str.getBytes());
-    Metainfo metainfo = new MetainfoParser().parse(metainfo_1);
+    Metainfo metainfo = new MetainfoParser().fromXmlStream(metainfo_1);
     Assert.assertNotNull(metainfo.getApplication());
     AgentProviderService aps = createAgentProviderService(new Configuration());
     HeartBeat hb = new HeartBeat();
@@ -964,7 +985,7 @@ public class TestAgentProviderService {
         new MockContainerId(1), "cid");
     AgentProviderService mockAps = Mockito.spy(aps);
     doNothing().when(mockAps).publishApplicationInstanceData(anyString(), anyString(), anyCollection());
-    doReturn(metainfo).when(mockAps).getMetainfo();
+    doReturn(metainfo).when(mockAps).getMetaInfo();
     doReturn(roleClusterNodeMap).when(mockAps).getRoleClusterNodeMapping();
     StateAccessForProviders access = createNiceMock(StateAccessForProviders.class);
     doReturn(access).when(mockAps).getAmState();
@@ -1018,9 +1039,9 @@ public class TestAgentProviderService {
   }
 
   @Test
-  public void testMetainfoParsing() throws Exception {
+  public void testMetaInfoParsing() throws Exception {
     InputStream metainfo_1 = new ByteArrayInputStream(metainfo_1_str.getBytes());
-    Metainfo metainfo = new MetainfoParser().parse(metainfo_1);
+    Metainfo metainfo = new MetainfoParser().fromXmlStream(metainfo_1);
     Assert.assertNotNull(metainfo.getApplication());
     Application application = metainfo.getApplication();
     log.info("Service: " + application.toString());
@@ -1031,32 +1052,23 @@ public class TestAgentProviderService {
     int found = 0;
     for (Component component : components) {
       if (component.getName().equals("HBASE_MASTER")) {
-        Assert.assertEquals(component.getAutoStartOnFailure(), "true");
-        Assert.assertEquals(component.getRequiresAutoRestart(), Boolean.TRUE);
-        Assert.assertEquals(component.getMinInstanceCount(), "1");
-        Assert.assertEquals(component.getMaxInstanceCount(), "2");
+        Assert.assertEquals("true", component.getAutoStartOnFailure());
+        Assert.assertEquals(Boolean.TRUE, component.getAutoStartOnFailureBoolean());
+        Assert.assertEquals(component.getMinInstanceCountInt(), 1);
+        Assert.assertEquals(component.getMaxInstanceCountInt(), 2);
         Assert.assertEquals(component.getCommandScript().getScript(), "scripts/hbase_master.py");
         Assert.assertEquals(component.getCategory(), "MASTER");
-        Assert.assertEquals(component.getComponentExports().size(), 0);
         Assert.assertEquals(component.getAppExports(), "QuickLinks-JMX_Endpoint,QuickLinks-Master_Status");
         Assert.assertEquals(component.getCompExports(), "QuickLinks-Comp_Endpoint");
         found++;
       }
       if (component.getName().equals("HBASE_REGIONSERVER")) {
-        Assert.assertEquals(component.getAutoStartOnFailure(), "Falsee");
-        Assert.assertEquals(component.getRequiresAutoRestart(), Boolean.FALSE);
+        Assert.assertEquals("Falsee", component.getAutoStartOnFailure());
+        Assert.assertEquals(Boolean.FALSE, component.getAutoStartOnFailureBoolean());
         Assert.assertEquals(component.getMinInstanceCount(), "1");
         Assert.assertNull(component.getMaxInstanceCount());
         Assert.assertEquals(component.getCommandScript().getScript(), "scripts/hbase_regionserver.py");
         Assert.assertEquals(component.getCategory(), "SLAVE");
-        Assert.assertEquals(component.getComponentExports().size(), 2);
-        List<ComponentExport> es = component.getComponentExports();
-        ComponentExport e = es.get(0);
-        Assert.assertEquals(e.getName(), "PropertyA");
-        Assert.assertEquals(e.getValue(), "${THIS_HOST}:${site.global.listen_port}");
-        e = es.get(1);
-        Assert.assertEquals(e.getName(), "PropertyB");
-        Assert.assertEquals(e.getValue(), "AConstant");
         found++;
       }
     }
@@ -1083,10 +1095,10 @@ public class TestAgentProviderService {
     }
     Assert.assertEquals(found, 2);
 
-    List<CommandOrder> cmdOrders = application.getCommandOrder();
+    List<CommandOrder> cmdOrders = application.getCommandOrders();
     Assert.assertEquals(cmdOrders.size(), 2);
     found = 0;
-    for (CommandOrder co : application.getCommandOrder()) {
+    for (CommandOrder co : application.getCommandOrders()) {
       if (co.getCommand().equals("HBASE_REGIONSERVER-START")) {
         Assert.assertTrue(co.getRequires().equals("HBASE_MASTER-STARTED"));
         found++;
@@ -1117,7 +1129,7 @@ public class TestAgentProviderService {
 
     AgentProviderService aps = createAgentProviderService(new Configuration());
     AgentProviderService mockAps = Mockito.spy(aps);
-    doReturn(metainfo).when(mockAps).getMetainfo();
+    doReturn(metainfo).when(mockAps).getMetaInfo();
     CommandScript script = mockAps.getScriptPathFromMetainfo("HBASE_MASTER");
     Assert.assertEquals(script.getScript(), "scripts/hbase_master.py");
 
@@ -1131,25 +1143,25 @@ public class TestAgentProviderService {
                                 + "      </comment>\n";
 
     metainfo_1 = new ByteArrayInputStream(metainfo_1_str_bad.getBytes());
-    metainfo = new MetainfoParser().parse(metainfo_1);
+    metainfo = new MetainfoParser().fromXmlStream(metainfo_1);
     Assert.assertNull(metainfo);
   }
 
   @Test
   public void testMetaInfoRelatedOperations() throws Exception {
     InputStream metainfo_1 = new ByteArrayInputStream(metainfo_1_str.getBytes());
-    Metainfo metainfo = new MetainfoParser().parse(metainfo_1);
+    Metainfo metainfo = new MetainfoParser().fromXmlStream(metainfo_1);
     InputStream metainfo_2 = new ByteArrayInputStream(metainfo_2_str.getBytes());
-    Metainfo metainfo2 = new MetainfoParser().parse(metainfo_2);
+    Metainfo metainfo2 = new MetainfoParser().fromXmlStream(metainfo_2);
     String role_hm = "HBASE_MASTER";
     String role_hrs = "HBASE_REGIONSERVER";
 
     AgentProviderService aps1 = createAgentProviderService(new Configuration());
     AgentProviderService mockAps = Mockito.spy(aps1);
-    doReturn(metainfo).when(mockAps).getMetainfo();
+    doReturn(metainfo).when(mockAps).getMetaInfo();
 
     AgentProviderService mockAps2 = Mockito.spy(aps1);
-    doReturn(metainfo2).when(mockAps2).getMetainfo();
+    doReturn(metainfo2).when(mockAps2).getMetaInfo();
 
     Assert.assertTrue(mockAps.isMaster(role_hm));
     Assert.assertFalse(mockAps.isMaster(role_hrs));
@@ -1165,11 +1177,11 @@ public class TestAgentProviderService {
   }
 
   @Test
-  public void testOrchestratedAppStart() throws IOException {
+  public void testOrchestratedAppStart() throws Exception {
     // App has two components HBASE_MASTER and HBASE_REGIONSERVER
     // Start of HBASE_RS depends on the start of HBASE_MASTER
     InputStream metainfo_1 = new ByteArrayInputStream(metainfo_1_str.getBytes());
-    Metainfo metainfo = new MetainfoParser().parse(metainfo_1);
+    Metainfo metainfo = new MetainfoParser().fromXmlStream(metainfo_1);
     ConfTree tree = new ConfTree();
     tree.global.put(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH, ".");
 
@@ -1222,12 +1234,15 @@ public class TestAgentProviderService {
           anyString(),
           any(HeartBeatResponse.class),
           anyString(),
+          eq((ComponentCommand)null),
           Mockito.anyLong());
       doNothing().when(mockAps).addStartCommand(
           anyString(),
           anyString(),
           any(HeartBeatResponse.class),
           anyString(),
+          eq((ComponentCommand)null),
+          eq((ComponentCommand)null),
           Mockito.anyLong(),
           Matchers.anyBoolean());
       doNothing().when(mockAps).addGetConfigCommand(
@@ -1308,6 +1323,7 @@ public class TestAgentProviderService {
                                                                   anyString(),
                                                                   any(HeartBeatResponse.class),
                                                                   anyString(),
+                                                                  eq((ComponentCommand)null),
                                                                   Mockito.anyLong());
 
       hb = new HeartBeat();
@@ -1319,6 +1335,7 @@ public class TestAgentProviderService {
                                                                   anyString(),
                                                                   any(HeartBeatResponse.class),
                                                                   anyString(),
+                                                                  eq((ComponentCommand)null),
                                                                   Mockito.anyLong());
       // RS succeeds install but does not start
       hb = new HeartBeat();
@@ -1338,6 +1355,8 @@ public class TestAgentProviderService {
                                                                 anyString(),
                                                                 any(HeartBeatResponse.class),
                                                                 anyString(),
+                                                                eq((ComponentCommand)null),
+                                                                eq((ComponentCommand)null),
                                                                 Mockito.anyLong(),
                                                                 Matchers.anyBoolean());
       // RS still does not start
@@ -1350,6 +1369,8 @@ public class TestAgentProviderService {
                                                                 anyString(),
                                                                 any(HeartBeatResponse.class),
                                                                 anyString(),
+                                                                eq((ComponentCommand)null),
+                                                                eq((ComponentCommand)null),
                                                                 Mockito.anyLong(),
                                                                 Matchers.anyBoolean());
 
@@ -1372,6 +1393,8 @@ public class TestAgentProviderService {
                                                                 anyString(),
                                                                 any(HeartBeatResponse.class),
                                                                 anyString(),
+                                                                eq((ComponentCommand)null),
+                                                                eq((ComponentCommand)null),
                                                                 Mockito.anyLong(),
                                                                 Matchers.anyBoolean());
       Map<String, String> allocatedPorts = mockAps.getAllocatedPorts();
@@ -1389,6 +1412,8 @@ public class TestAgentProviderService {
                                                                 anyString(),
                                                                 any(HeartBeatResponse.class),
                                                                 anyString(),
+                                                                eq((ComponentCommand)null),
+                                                                eq((ComponentCommand)null),
                                                                 Mockito.anyLong(),
                                                                 Matchers.anyBoolean());
       // MASTER succeeds start
@@ -1415,6 +1440,8 @@ public class TestAgentProviderService {
                                                                 anyString(),
                                                                 any(HeartBeatResponse.class),
                                                                 anyString(),
+                                                                eq((ComponentCommand)null),
+                                                                eq((ComponentCommand)null),
                                                                 Mockito.anyLong(),
                                                                 Matchers.anyBoolean());
     // JDK7 
@@ -1542,7 +1569,7 @@ public class TestAgentProviderService {
   @Test
   public void testAddInstallCommand() throws Exception {
     InputStream metainfo_1 = new ByteArrayInputStream(metainfo_1_str.getBytes());
-    Metainfo metainfo = new MetainfoParser().parse(metainfo_1);
+    Metainfo metainfo = new MetainfoParser().fromXmlStream(metainfo_1);
     AgentProviderService aps = createAgentProviderService(new Configuration());
     HeartBeatResponse hbr = new HeartBeatResponse();
 
@@ -1563,7 +1590,7 @@ public class TestAgentProviderService {
     expect(access.isApplicationLive()).andReturn(true).anyTimes();
 
     doReturn("HOST1").when(mockAps).getClusterInfoPropertyValue(anyString());
-    doReturn(metainfo).when(mockAps).getMetainfo();
+    doReturn(metainfo).when(mockAps).getMetaInfo();
     doReturn(new HashMap<String, DefaultConfig>()).when(mockAps).getDefaultConfigs();
 
     Map<String, Map<String, ClusterNode>> roleClusterNodeMap = new HashMap<String, Map<String, ClusterNode>>();
@@ -1576,7 +1603,7 @@ public class TestAgentProviderService {
 
     replay(access);
 
-    mockAps.addInstallCommand("HBASE_MASTER", "cid1", hbr, "", 0);
+    mockAps.addInstallCommand("HBASE_MASTER", "cid1", hbr, "", null, 0);
     ExecutionCommand cmd = hbr.getExecutionCommands().get(0);
     String pkgs = cmd.getHostLevelParams().get(AgentKeys.PACKAGE_LIST);
     Assert.assertEquals("[{\"type\":\"tarball\",\"name\":\"files/hbase-0.96.1-hadoop2-bin.tar.gz\"}]", pkgs);
@@ -1656,7 +1683,9 @@ public class TestAgentProviderService {
 
     replay(access);
 
-    mockAps.addStartCommand("HBASE_MASTER", "cid1", hbr, "", 0, Boolean.FALSE);
+    ComponentCommand startCmd = ComponentCommand.getDefaultComponentCommand();
+    ComponentCommand stopCmd = ComponentCommand.getDefaultComponentCommand("STOP");
+    mockAps.addStartCommand("HBASE_MASTER", "cid1", hbr, "", startCmd, stopCmd, 0, Boolean.FALSE);
     Assert.assertTrue(hbr.getExecutionCommands().get(0).getConfigurations().containsKey("hbase-site"));
     Assert.assertTrue(hbr.getExecutionCommands().get(0).getConfigurations().containsKey("core-site"));
     Map<String, String> hbaseSiteConf = hbr.getExecutionCommands().get(0).getConfigurations().get("hbase-site");

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java b/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
index 1177e9d..30283d1 100644
--- a/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
+++ b/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
@@ -45,7 +45,7 @@ public class TestMetainfoParser {
     InputStream resStream = this.getClass().getResourceAsStream(
         METAINFO_XML);
     MetainfoParser parser = new MetainfoParser();
-    Metainfo metainfo = parser.parse(resStream);
+    Metainfo metainfo = parser.fromXmlStream(resStream);
     Assert.assertNotNull(metainfo);
     Assert.assertNotNull(metainfo.getApplication());
     Application application = metainfo.getApplication();
@@ -67,4 +67,100 @@ public class TestMetainfoParser {
     assert found;
     Assert.assertEquals(0, application.getConfigFiles().size());
   }
+
+  @Test
+  public void testJsonParse() throws IOException {
+    String metaInfo1_json = "{\n"
+                            + "\"schemaVersion\":\"2.2\",\n"
+                            + "\"application\":{\n"
+                            +     "\"name\": \"MEMCACHED\","
+                            +     "\"exportGroups\": ["
+                            +        "{"
+                            +          "\"name\": \"Servers\","
+                            +          "\"exports\": ["
+                            +            "{"
+                            +               "\"name\": \"host_port\","
+                            +               "\"value\": \"${MEMCACHED_HOST}:${site.global.port}\""
+                            +            "}"
+                            +          "]"
+                            +        "}"
+                            +      "],"
+                            +     "\"components\": ["
+                            +        "{"
+                            +          "\"name\": \"MEMCACHED\","
+                            +          "\"compExports\": \"Servers-host_port\","
+                            +          "\"commands\": ["
+                            +            "{"
+                            +               "\"exec\": \"java -classpath /usr/myapps/memcached/*:/usr/lib/hadoop/lib/* com.thimbleware.jmemcached.Main\""
+                            +            "}"
+                            +          "]"
+                            +        "},"
+                            +        "{"
+                            +          "\"name\": \"MEMCACHED2\","
+                            +          "\"commands\": ["
+                            +            "{"
+                            +               "\"exec\": \"scripts/config.py\","
+                            +               "\"type\": \"PYTHON\","
+                            +               "\"name\": \"CONFIGURE\""
+                            +            "}"
+                            +          "],"
+                            +          "\"dockerContainers\": ["
+                            +            "{"
+                            +               "\"name\": \"redis\","
+                            +               "\"image\": \"dockerhub/redis\","
+                            +               "\"options\": \"-net=bridge\","
+                            +               "\"mounts\": ["
+                            +                 "{"
+                            +                   "\"containerMount\": \"/tmp/conf\","
+                            +                   "\"hostMount\": \"{$conf:@//site/global/app_root}/conf\""
+                            +                 "}"
+                            +               "]"
+                            +            "}"
+                            +          "]"
+                            +        "}"
+                            +      "]"
+                            +   "}"
+                            + "}";
+
+    MetainfoParser parser = new MetainfoParser();
+    Metainfo mInfo = parser.fromJsonString(metaInfo1_json);
+    Assert.assertEquals("2.2", mInfo.getSchemaVersion());
+
+    Application app = mInfo.getApplication();
+    Assert.assertNotNull(app);
+
+    Assert.assertEquals("MEMCACHED", app.getName());
+    List<ExportGroup> egs = app.getExportGroups();
+    Assert.assertEquals(1, egs.size());
+    ExportGroup eg = egs.get(0);
+    Assert.assertEquals("Servers", eg.getName());
+    List<Export> exports = eg.getExports();
+    Assert.assertEquals(1, exports.size());
+    Export export = exports.get(0);
+    Assert.assertEquals("host_port", export.getName());
+    Assert.assertEquals("${MEMCACHED_HOST}:${site.global.port}", export.getValue());
+
+    List<Component> components = app.getComponents();
+    Assert.assertEquals(2, components.size());
+
+    Component c1 = mInfo.getApplicationComponent("MEMCACHED");
+    Assert.assertNotNull(c1);
+    Assert.assertEquals("MEMCACHED", c1.getName());
+    Assert.assertEquals("Servers-host_port", c1.getCompExports());
+    Assert.assertEquals(1, c1.getCommands().size());
+    ComponentCommand cmd = c1.getCommands().get(0);
+    Assert.assertEquals("START", cmd.getName());
+    Assert.assertEquals("SHELL", cmd.getType());
+    Assert.assertEquals("java -classpath /usr/myapps/memcached/*:/usr/lib/hadoop/lib/* com.thimbleware.jmemcached.Main",
+                        cmd.getExec());
+
+    Component c2 = mInfo.getApplicationComponent("MEMCACHED2");
+    Assert.assertNotNull(c2);
+    Assert.assertEquals("MEMCACHED2", c2.getName());
+    Assert.assertEquals(1, c2.getCommands().size());
+    cmd = c2.getCommands().get(0);
+    Assert.assertEquals("CONFIGURE", cmd.getName());
+    Assert.assertEquals("PYTHON", cmd.getType());
+    Assert.assertEquals("scripts/config.py", cmd.getExec());
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
index 95021b8..1c94eae 100644
--- a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
@@ -19,6 +19,7 @@
 package org.apache.slider.funtest.framework
 
 import groovy.transform.CompileStatic
+import org.apache.commons.lang.StringUtils
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.FileSystem as HadoopFS
 import org.apache.hadoop.fs.Path
@@ -622,6 +623,85 @@ abstract class CommandTestBase extends SliderTestUtils {
   }
 
   /**
+   * Create a slider app using the alternate packaging capability
+   * <p>
+   * If the extraArgs list does not contain a --wait parm then a wait
+   * duration of THAW_WAIT_TIME will be added to the launch args.
+   * @param name name
+   * @param metaInfo application metaInfo
+   * @param appTemplate application template
+   * @param resourceTemplate resource template
+   * @param extraArgs list of extra arguments to the command
+   * @param launchReportFile optional file to save the AM launch report to
+   * @return the shell
+   */
+  public SliderShell createSliderApplicationMinPkg(
+      String name,
+      String metaInfo,
+      String resourceTemplate,
+      String appTemplate,
+      List<String> extraArgs = [],
+      File launchReportFile = null) {
+
+    if (!launchReportFile) {
+      launchReportFile = createTempJsonFile()
+    }
+    // delete any previous copy of the file
+    launchReportFile.delete();
+
+    List<String> commands = [
+        ACTION_CREATE, name,
+        ARG_METAINFO, metaInfo,
+        ARG_OUTPUT, launchReportFile.absolutePath
+    ]
+
+    if (StringUtils.isNotBlank(appTemplate)) {
+      commands << ARG_TEMPLATE << appTemplate
+    }
+    if (StringUtils.isNotBlank(resourceTemplate)) {
+      commands << ARG_RESOURCES << resourceTemplate
+    }
+    if (!extraArgs.contains(ARG_WAIT)) {
+      commands << ARG_WAIT << Integer.toString(THAW_WAIT_TIME)
+    }
+
+    maybeAddCommandOption(commands,
+        [ARG_COMP_OPT, SliderKeys.COMPONENT_AM, SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME],
+        SLIDER_CONFIG.getTrimmed(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME));
+    maybeAddCommandOption(commands,
+        [ARG_COMP_OPT, SliderKeys.COMPONENT_AM, SliderXmlConfKeys.KEY_HDFS_KEYTAB_DIR],
+        SLIDER_CONFIG.getTrimmed(SliderXmlConfKeys.KEY_HDFS_KEYTAB_DIR));
+    maybeAddCommandOption(commands,
+        [ARG_COMP_OPT, SliderKeys.COMPONENT_AM, SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH],
+        SLIDER_CONFIG.getTrimmed(SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH));
+    maybeAddCommandOption(commands,
+        [ARG_COMP_OPT, SliderKeys.COMPONENT_AM, SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL],
+        SLIDER_CONFIG.getTrimmed(SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL));
+    commands.addAll(extraArgs)
+    SliderShell shell = new SliderShell(commands)
+    if (0 != shell.execute()) {
+      // app has failed.
+
+      // grab the app report of the last known instance of this app
+      // which may not be there if it was a config failure; may be out of date
+      // from a previous run
+      log.error("Launch failed with exit code ${shell.ret}")
+      shell.dumpOutput()
+
+      // now grab that app report if it is there
+      def appReport = maybeLookupFromLaunchReport(launchReportFile)
+      String extraText = ""
+      if (appReport) {
+        log.error("Application report:\n$appReport")
+        extraText = appReport.diagnostics
+      }
+
+      fail("Application Launch Failure, exit code  ${shell.ret}\n${extraText}")
+    }
+    return shell
+  }
+
+  /**
    * Create a templated slider app.
    * <p>
    * If the extraArgs list does not contain a --wait parm then a wait 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentMinSleepIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentMinSleepIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentMinSleepIT.groovy
new file mode 100644
index 0000000..e7b0454
--- /dev/null
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentMinSleepIT.groovy
@@ -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.slider.funtest.lifecycle
+
+import groovy.transform.CompileStatic
+import groovy.util.logging.Slf4j
+import org.apache.hadoop.registry.client.binding.RegistryUtils
+import org.apache.hadoop.registry.client.types.Endpoint
+import org.apache.hadoop.registry.client.types.ServiceRecord
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
+import org.apache.slider.common.SliderExitCodes
+import org.apache.slider.common.SliderKeys
+import org.apache.slider.common.SliderXmlConfKeys
+import org.apache.slider.common.params.Arguments
+import org.apache.slider.common.params.SliderActions
+import org.apache.slider.test.Outcome
+
+import static org.apache.slider.core.registry.info.CustomRegistryConstants.*
+import org.apache.slider.funtest.framework.AgentCommandTestBase
+import org.apache.slider.funtest.framework.FuntestProperties
+import org.apache.slider.funtest.framework.SliderShell
+import org.junit.After
+import org.junit.Before
+import org.junit.Test
+
+@CompileStatic
+@Slf4j
+public class AgentMinSleepIT extends AgentCommandTestBase
+    implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
+
+
+  static String CLUSTER = "test-agent-sleep-100"
+
+  static String APP_RESOURCE11 = "../slider-core/src/test/app_packages/test_min_pkg/sleep_cmd/resources.json"
+  static String APP_META11 = "../slider-core/src/test/app_packages/test_min_pkg/sleep_cmd/metainfo.json"
+
+
+  @Before
+  public void prepareCluster() {
+    setupCluster(CLUSTER)
+  }
+
+  @After
+  public void destroyCluster() {
+    cleanup(CLUSTER)
+  }
+
+  @Test
+  public void testAgentRegistry() throws Throwable {
+    describe("Create a cluster using metainfo and resources only that executes sleep 100")
+    def clusterpath = buildClusterPath(CLUSTER)
+    File launchReportFile = createTempJsonFile();
+
+    SliderShell shell = createSliderApplicationMinPkg(CLUSTER,
+        APP_META11,
+        APP_RESOURCE11,
+        null,
+        [],
+        launchReportFile)
+
+    logShell(shell)
+
+    def appId = ensureYarnApplicationIsUp(launchReportFile)
+
+    //at this point the cluster should exist.
+    assertPathExists(
+        clusterFS,
+        "Cluster parent directory does not exist",
+        clusterpath.parent)
+
+    assertPathExists(clusterFS, "Cluster directory does not exist", clusterpath)
+
+    status(0, CLUSTER)
+    expectLiveContainerCountReached(CLUSTER, "SLEEP_100", 1,
+        CONTAINER_LAUNCH_TIMEOUT)
+
+    // sleep for some manual test
+    describe("You may quickly perform manual tests against the application instance " + CLUSTER)
+    sleep(1000 * 30)
+
+    //stop
+    freeze(0, CLUSTER,
+        [
+            ARG_WAIT, Integer.toString(FREEZE_WAIT_TIME),
+            ARG_MESSAGE, "final-shutdown"
+        ])
+
+    assertInYarnState(appId, YarnApplicationState.FINISHED)
+    destroy(0, CLUSTER)
+
+    //cluster now missing
+    exists(EXIT_UNKNOWN_INSTANCE, CLUSTER)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/704e8136/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentPingSocketIT.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentPingSocketIT.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentPingSocketIT.groovy
new file mode 100644
index 0000000..0d8724f
--- /dev/null
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentPingSocketIT.groovy
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.funtest.lifecycle
+
+import groovy.transform.CompileStatic
+import groovy.util.logging.Slf4j
+import groovy.json.*
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.registry.client.binding.RegistryUtils
+import org.apache.hadoop.registry.client.types.Endpoint
+import org.apache.hadoop.registry.client.types.ServiceRecord
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
+import org.apache.slider.common.SliderExitCodes
+import org.apache.slider.common.SliderKeys
+import org.apache.slider.common.SliderXmlConfKeys
+import org.apache.slider.common.params.Arguments
+import org.apache.slider.common.params.SliderActions
+import org.apache.slider.test.Outcome
+
+import static org.apache.slider.core.registry.info.CustomRegistryConstants.*
+import org.apache.slider.funtest.framework.AgentCommandTestBase
+import org.apache.slider.funtest.framework.FuntestProperties
+import org.apache.slider.funtest.framework.SliderShell
+import org.junit.After
+import org.junit.Before
+import org.junit.Test
+
+@CompileStatic
+@Slf4j
+public class AgentPingSocketIT extends AgentCommandTestBase
+    implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
+
+
+  static String CLUSTER = "test-agent-ping-port"
+
+  static String APP_RESOURCE12 = "../slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/resources.json"
+  static String APP_META12 = "../slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/metainfo.json"
+  static String APP_TEMPLATE12 = "../slider-core/src/test/app_packages/test_min_pkg/nc_ping_cmd/appConfig.json"
+
+
+  @Before
+  public void prepareCluster() {
+    setupCluster(CLUSTER)
+  }
+
+  @After
+  public void destroyCluster() {
+    cleanup(CLUSTER)
+  }
+
+  @Test
+  public void testAgentRegistry() throws Throwable {
+    describe("Create a cluster using metainfo, resources, and appConfig that calls nc to listen on a port")
+    def clusterpath = buildClusterPath(CLUSTER)
+    File launchReportFile = createTempJsonFile();
+
+    SliderShell shell = createSliderApplicationMinPkg(CLUSTER,
+        APP_META12,
+        APP_RESOURCE12,
+        APP_TEMPLATE12,
+        [],
+        launchReportFile)
+
+    logShell(shell)
+
+    def appId = ensureYarnApplicationIsUp(launchReportFile)
+
+    describe("Checking the exported port value and pinging it for " + CLUSTER)
+    def outfile = tmpFile(".txt")
+
+    def commands = [
+        ACTION_REGISTRY,
+        ARG_NAME,
+        CLUSTER,
+        ARG_LISTEXP,
+        ARG_OUTPUT,
+        outfile.absolutePath
+    ]
+
+    awaitRegistryOutfileContains(outfile, commands, "servers")
+
+    // get Servers host_port folders
+    slider(EXIT_SUCCESS,
+        [
+            ACTION_REGISTRY,
+            ARG_NAME,
+            CLUSTER,
+            ARG_GETEXP,
+            "servers",
+            ARG_OUTPUT,
+            outfile.absolutePath])
+
+    describe(outfile.absolutePath)
+
+    def result = new JsonSlurper().parseText(outfile.text)
+    Map jsonResult = (Map) result
+    List host_ports = (List)jsonResult.get("host_port")
+    Map host_port = (Map)host_ports[0]
+    String host_port_val = host_port.get("value")
+    def tokens = host_port_val.tokenize(':')
+    def host = tokens[0]
+    def port = tokens[1].toInteger()
+
+    try {
+      def socket = new Socket();
+      def addr = new InetSocketAddress(host, port)
+      socket.connect(addr, 2000)
+      socket.close()
+    } catch (IOException e) {
+      throw NetUtils.wrapException(host, port, "localhost", 0, e)
+    }
+
+    //stop
+    freeze(0, CLUSTER,
+        [
+            ARG_WAIT, Integer.toString(FREEZE_WAIT_TIME),
+            ARG_MESSAGE, "final-shutdown"
+        ])
+
+    assertInYarnState(appId, YarnApplicationState.FINISHED)
+    destroy(0, CLUSTER)
+
+    //cluster now missing
+    exists(EXIT_UNKNOWN_INSTANCE, CLUSTER)
+  }
+}