You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by st...@apache.org on 2014/05/15 14:42:52 UTC

svn commit: r1594902 - in /incubator/slider/trunk: slider-core/src/main/java/org/apache/slider/client/ slider-core/src/main/java/org/apache/slider/common/params/ slider-core/src/main/java/org/apache/slider/core/registry/info/ slider-core/src/main/java/...

Author: stevel
Date: Thu May 15 12:42:51 2014
New Revision: 1594902

URL: http://svn.apache.org/r1594902
Log:
SLIDER-5 CLI to list and retrieve service bindings & configs
this patch sets up the docs and tests ready for a deployment

Added:
    incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseClusterBuildDestroy.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestClusterBuildDestroy.groovy
    incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseClusterLifecycle.groovy
      - copied, changed from r1594525, incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestClusterLifecycle.groovy
Removed:
    incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestClusterBuildDestroy.groovy
    incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestClusterLifecycle.groovy
Modified:
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/ActionRegistryArgs.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/RegisteredEndpoint.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/ServiceInstanceData.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy
    incubator/slider/trunk/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
    incubator/slider/trunk/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy
    incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterLifecycle.groovy
    incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestFunctionalHBaseCluster.groovy
    incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseIntegration.groovy
    incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseLoad.groovy
    incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseNodeFailure.groovy
    incubator/slider/trunk/src/site/markdown/manpage.md

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/client/SliderClient.java?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/client/SliderClient.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/client/SliderClient.java Thu May 15 12:42:51 2014
@@ -55,7 +55,6 @@ import org.apache.slider.common.params.A
 import org.apache.slider.common.params.Arguments;
 import org.apache.slider.common.params.ClientArgs;
 import org.apache.slider.common.params.LaunchArgsAccessor;
-import org.apache.slider.common.params.SliderActions;
 import org.apache.slider.common.tools.ConfigHelper;
 import org.apache.slider.common.tools.Duration;
 import org.apache.slider.common.tools.SliderFileSystem;
@@ -91,6 +90,7 @@ import org.apache.slider.core.registry.d
 import org.apache.slider.core.registry.docstore.PublishedConfigSet;
 import org.apache.slider.core.registry.docstore.PublishedConfiguration;
 import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
+import org.apache.slider.core.registry.info.RegisteredEndpoint;
 import org.apache.slider.core.registry.info.ServiceInstanceData;
 import org.apache.slider.core.registry.retrieve.RegistryRetriever;
 import org.apache.slider.core.registry.zk.ZKPathBuilder;
@@ -101,9 +101,10 @@ import org.apache.slider.providers.slide
 import org.apache.slider.server.appmaster.SliderAppMaster;
 import org.apache.slider.server.appmaster.rpc.RpcBinder;
 import org.apache.slider.server.services.curator.CuratorServiceInstance;
-import org.apache.slider.server.services.curator.RegistryBinderService;
 import org.apache.slider.server.services.registry.SliderRegistryService;
 import org.apache.slider.server.services.utility.AbstractSliderLaunchedService;
+
+import static org.apache.slider.common.params.SliderActions.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -114,6 +115,7 @@ import java.io.IOException;
 import java.io.StringWriter;
 import java.io.Writer;
 import java.net.InetSocketAddress;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -201,50 +203,62 @@ public class SliderClient extends Abstra
     int exitCode = EXIT_SUCCESS;
     String clusterName = serviceArgs.getClusterName();
     // actions
-    if (SliderActions.ACTION_BUILD.equals(action)) {
-      exitCode = actionBuild(clusterName, serviceArgs.getActionBuildArgs());
-    } else if (SliderActions.ACTION_CREATE.equals(action)) {
-      exitCode = actionCreate(clusterName, serviceArgs.getActionCreateArgs());
-    } else if (SliderActions.ACTION_FREEZE.equals(action)) {
-      exitCode = actionFreeze(clusterName,
-                              serviceArgs.getActionFreezeArgs());
-    } else if (SliderActions.ACTION_THAW.equals(action)) {
-      exitCode = actionThaw(clusterName, serviceArgs.getActionThawArgs());
-    } else if (SliderActions.ACTION_DESTROY.equals(action)) {
-      exitCode = actionDestroy(clusterName);
-    } else if (SliderActions.ACTION_EXISTS.equals(action)) {
-      exitCode = actionExists(clusterName,
-                              serviceArgs.getActionExistsArgs().live);
-    } else if (SliderActions.ACTION_FLEX.equals(action)) {
-      exitCode = actionFlex(clusterName, serviceArgs.getActionFlexArgs());
-    } else if (SliderActions.ACTION_GETCONF.equals(action)) {
-      exitCode = actionGetConf(clusterName, serviceArgs.getActionGetConfArgs());
-    } else if (SliderActions.ACTION_HELP.equals(action) ||
-               SliderActions.ACTION_USAGE.equals(action)) {
-      log.info(serviceArgs.usage());
-
-    } else if (SliderActions.ACTION_KILL_CONTAINER.equals(action)) {
-      exitCode = actionKillContainer(clusterName,
-                                     serviceArgs.getActionKillContainerArgs());
-
-    } else if (SliderActions.ACTION_AM_SUICIDE.equals(action)) {
-      exitCode = actionAmSuicide(clusterName,
-                                 serviceArgs.getActionAMSuicideArgs());
-
-    } else if (SliderActions.ACTION_LIST.equals(action)) {
-      exitCode = actionList(clusterName);
-    } else if (SliderActions.ACTION_REGISTRY.equals(action)) {     
-      exitCode = actionRegistry(
-          serviceArgs.getActionRegistryArgs());
-    } else if (SliderActions.ACTION_STATUS.equals(action)) {     
-      exitCode = actionStatus(clusterName,
-                              serviceArgs.getActionStatusArgs());
-    } else if (SliderActions.ACTION_VERSION.equals(action)) {
-      
-      exitCode = actionVersion();
-    } else {
-      throw new SliderException(EXIT_UNIMPLEMENTED,
-                              "Unimplemented: " + action);
+    switch (action) {
+      case ACTION_BUILD:
+        exitCode = actionBuild(clusterName, serviceArgs.getActionBuildArgs());
+        break;
+      case ACTION_CREATE:
+        exitCode = actionCreate(clusterName, serviceArgs.getActionCreateArgs());
+        break;
+      case ACTION_FREEZE:
+        exitCode = actionFreeze(clusterName, serviceArgs.getActionFreezeArgs());
+        break;
+      case ACTION_THAW:
+        exitCode = actionThaw(clusterName, serviceArgs.getActionThawArgs());
+        break;
+      case ACTION_DESTROY:
+        exitCode = actionDestroy(clusterName);
+        break;
+      case ACTION_EXISTS:
+        exitCode = actionExists(clusterName,
+            serviceArgs.getActionExistsArgs().live);
+        break;
+      case ACTION_FLEX:
+        exitCode = actionFlex(clusterName, serviceArgs.getActionFlexArgs());
+        break;
+      case ACTION_GETCONF:
+        exitCode =
+            actionGetConf(clusterName, serviceArgs.getActionGetConfArgs());
+        break;
+      case ACTION_HELP:
+      case ACTION_USAGE:
+        log.info(serviceArgs.usage());
+        break;
+      case ACTION_KILL_CONTAINER:
+        exitCode = actionKillContainer(clusterName,
+            serviceArgs.getActionKillContainerArgs());
+        break;
+      case ACTION_AM_SUICIDE:
+        exitCode = actionAmSuicide(clusterName,
+            serviceArgs.getActionAMSuicideArgs());
+        break;
+      case ACTION_LIST:
+        exitCode = actionList(clusterName);
+        break;
+      case ACTION_REGISTRY:
+        actionRegistry(
+            serviceArgs.getActionRegistryArgs());
+        break;
+      case ACTION_STATUS:
+        exitCode = actionStatus(clusterName,
+            serviceArgs.getActionStatusArgs());
+        break;
+      case ACTION_VERSION:
+        exitCode = actionVersion();
+        break;
+      default:
+        throw new SliderException(EXIT_UNIMPLEMENTED,
+            "Unimplemented: " + action);
     }
 
     return exitCode;
@@ -567,7 +581,7 @@ public class SliderClient extends Abstra
 
   /**
    * Load the instance definition. It is not resolved at this point
-   * @param name
+   * @param name cluster name
    * @param clusterDirectory cluster dir
    * @return the loaded configuration
    * @throws IOException
@@ -590,7 +604,7 @@ public class SliderClient extends Abstra
   }
     /**
    * Load the instance definition. 
-   * @param name
+   * @param name cluster name
    * @param resolved flag to indicate the cluster should be resolved
    * @return the loaded configuration
    * @throws IOException
@@ -823,7 +837,7 @@ public class SliderClient extends Abstra
 
     // rm address
 
-    InetSocketAddress rmSchedulerAddress = null;
+    InetSocketAddress rmSchedulerAddress;
     try {
       rmSchedulerAddress = SliderUtils.getRmSchedulerAddress(config);
     } catch (IllegalArgumentException e) {
@@ -845,7 +859,7 @@ public class SliderClient extends Abstra
     commandLine.add(SliderAppMaster.SERVICE_CLASSNAME);
 
     // create action and the cluster name
-    commandLine.add(SliderActions.ACTION_CREATE, clustername);
+    commandLine.add(ACTION_CREATE, clustername);
 
     // debug
     if (debugAM) {
@@ -962,22 +976,6 @@ public class SliderClient extends Abstra
 
   /**
    * Propagate any critical principals from the current site config down to the HBase one.
-   * @param clusterSpec cluster spec
-   * @param config config to read from
-   */
-  private void propagatePrincipals(ClusterDescription clusterSpec,
-                                   Configuration config) {
-    String dfsPrincipal = config.get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY);
-    if (dfsPrincipal != null) {
-      String siteDfsPrincipal = OptionKeys.SITE_XML_PREFIX +
-                                DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY;
-      clusterSpec.setOptionifUnset(siteDfsPrincipal, dfsPrincipal);
-    }
-  }
-
-
-  /**
-   * Propagate any critical principals from the current site config down to the HBase one.
    * @param config config to read from
    * @param clusterSpec cluster spec
    */
@@ -1249,7 +1247,7 @@ public class SliderClient extends Abstra
     verifyBindingsDefined();
     SliderUtils.validateClusterName(name);
     log.debug("actionFlex({})", name);
-    Map<String, Integer> roleInstances = new HashMap<String, Integer>();
+    Map<String, Integer> roleInstances = new HashMap<>();
     Map<String, String> roleMap = args.getComponentMap();
     for (Map.Entry<String, String> roleEntry : roleMap.entrySet()) {
       String key = roleEntry.getKey();
@@ -1544,10 +1542,7 @@ public class SliderClient extends Abstra
           return EXIT_FALSE;
         }
       }
-    } catch (YarnException e) {
-      log.warn("Exception while waiting for the cluster {} to shut down: {}",
-               clustername, e);
-    } catch (IOException e) {
+    } catch (YarnException | IOException e) {
       log.warn("Exception while waiting for the cluster {} to shut down: {}",
                clustername, e);
     }
@@ -1611,15 +1606,18 @@ public class SliderClient extends Abstra
     }
     try {
       String description = "Slider Application Instance " + clustername;
-      if (format.equals(Arguments.FORMAT_XML)) {
-        Configuration siteConf = getSiteConf(status, clustername);
-        siteConf.writeXml(writer);
-      } else if (format.equals(Arguments.FORMAT_PROPERTIES)) {
-        Properties props = new Properties();
-        props.putAll(status.clientProperties);
-        props.store(writer, description);
-      } else {
-        throw new BadCommandArgumentsException("Unknown format: " + format);
+      switch (format) {
+        case Arguments.FORMAT_XML:
+          Configuration siteConf = getSiteConf(status, clustername);
+          siteConf.writeXml(writer);
+          break;
+        case Arguments.FORMAT_PROPERTIES:
+          Properties props = new Properties();
+          props.putAll(status.clientProperties);
+          props.store(writer, description);
+          break;
+        default:
+          throw new BadCommandArgumentsException("Unknown format: " + format);
       }
     } finally {
       // data is written.
@@ -1778,7 +1776,7 @@ public class SliderClient extends Abstra
   /**
    * List all nodes in a role. This is a double round trip: once to list
    * the nodes in a role, another to get their details
-   * @param role
+   * @param role component/role to look for
    * @return an array of ContainerNode instances
    * @throws IOException
    * @throws YarnException
@@ -1792,7 +1790,7 @@ public class SliderClient extends Abstra
 
   /**
    * Get the details on a list of uuids
-   * @param uuids
+   * @param uuids uuids to ask for 
    * @return a possibly empty list of node details
    * @throws IOException
    * @throws YarnException
@@ -1804,7 +1802,7 @@ public class SliderClient extends Abstra
 
     if (uuids.length == 0) {
       // short cut on an empty list
-      return new LinkedList<ClusterNode>();
+      return new LinkedList<>();
     }
     return createClusterOperations().listClusterNodes(uuids);
   }
@@ -1947,13 +1945,12 @@ public class SliderClient extends Abstra
    * @throws IOException Network or other problems
    */
   @VisibleForTesting
-  public int actionRegistry(ActionRegistryArgs registryArgs) throws
+  public void actionRegistry(ActionRegistryArgs registryArgs) throws
       YarnException,
       IOException {
     // as this is also a test entry point, validate
     // the arguments
     registryArgs.validate();
-    int exitCode = EXIT_SUCCESS;
     if (registryArgs.list) {
       actionRegistryList(registryArgs);
     } else if (registryArgs.listConf) {
@@ -1961,34 +1958,64 @@ public class SliderClient extends Abstra
       actionRegistryListConfigs(registryArgs);
     } else if (SliderUtils.isSet(registryArgs.getConf)) {
       // get a configuration
-      PublishedConfiguration publishedConfiguration =
-          actionRegistryGetConfig(registryArgs);
-      outputConfig(publishedConfiguration, registryArgs);
-      
+      try {
+        PublishedConfiguration publishedConfiguration =
+            actionRegistryGetConfig(registryArgs);
+        outputConfig(publishedConfiguration, registryArgs);
+      } catch (FileNotFoundException e) {
+// TODO
+      }
+
     } else {
-      exitCode = EXIT_FALSE;
+      // its an unknown command
+      throw new BadCommandArgumentsException(
+          "Bad command arguments for "+ ACTION_REGISTRY +" " + registryArgs);
     }
-    return exitCode;
   }
 
   /**
    * Registry operation
    *
    * @param registryArgs registry Arguments
+   * @return the instances (for tests)
    * @throws YarnException YARN problems
    * @throws IOException Network or other problems
    */
-  private void actionRegistryList(ActionRegistryArgs registryArgs)
+  @VisibleForTesting
+  public List<ServiceInstanceData> actionRegistryList(
+      ActionRegistryArgs registryArgs)
       throws YarnException, IOException {
+    SliderRegistryService registryService = getRegistry();
     List<CuratorServiceInstance<ServiceInstanceData>> instances =
-        getRegistry().listInstances(registryArgs.serviceType);
-
+        registryService.findInstances(registryArgs.serviceType, registryArgs.name
+        );
+    List<ServiceInstanceData> sids = new ArrayList<>(instances.size());
     for (CuratorServiceInstance<ServiceInstanceData> instance : instances) {
-      if (!registryArgs.verbose) {
-        log.info("{}", instance.id);
-      } else {
-        log.info("{} ", instance);
-      }
+      ServiceInstanceData payload = instance.payload;
+      
+      logInstance(payload, registryArgs.verbose);
+      sids.add(payload);
+    }
+    return sids;
+  }
+
+  private void logInstance(ServiceInstanceData instance,
+      boolean verbose) {
+    if (!verbose) {
+      log.info("{}", instance.id);
+    } else {
+      log.info("{}: ", instance.id);
+      logEndpoints(instance);
+    }
+  }
+  
+  private void logEndpoints(ServiceInstanceData instance) {
+      Map<String, RegisteredEndpoint> endpoints =
+          instance.listEndpoints(true);
+      for (Map.Entry<String, RegisteredEndpoint> entry : endpoints.entrySet()) {
+        String name = entry.getKey();
+        RegisteredEndpoint endpoint = entry.getValue();
+        log.info("  {}", endpoint);
     }
   }
 
@@ -2026,7 +2053,9 @@ public class SliderClient extends Abstra
    * @param registryArgs registry Arguments
    * @throws YarnException YARN problems
    * @throws IOException Network or other problems
+   * @throws FileNotFoundException if the config is not found
    */
+  @VisibleForTesting
   public PublishedConfiguration actionRegistryGetConfig(ActionRegistryArgs registryArgs)
       throws YarnException, IOException {
     ServiceInstanceData instance = lookupInstance(registryArgs);
@@ -2036,8 +2065,9 @@ public class SliderClient extends Abstra
     PublishedConfigSet configurations =
         retriever.getConfigurations(external);
 
-    PublishedConfiguration published =
-        retriever.retrieveConfiguration(registryArgs.getConf, external);
+    PublishedConfiguration published = retriever.retrieveConfiguration(configurations,
+            registryArgs.getConf,
+            external);
     return published;
   }
   
@@ -2130,7 +2160,7 @@ public class SliderClient extends Abstra
 
   /**
    * List instances in the registry
-   * @return
+   * @return the instance IDs
    * @throws IOException
    * @throws YarnException
    */

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/ActionRegistryArgs.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/ActionRegistryArgs.java?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/ActionRegistryArgs.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/ActionRegistryArgs.java Thu May 15 12:42:51 2014
@@ -80,7 +80,7 @@ public class ActionRegistryArgs extends 
 
   //--format 
   @Parameter(names = ARG_FORMAT,
-      description = "Format for a response: [text|xml|json|properties]")
+      description = "Format for a response: [xml|json|properties]")
   public String format;
 
 
@@ -96,7 +96,6 @@ public class ActionRegistryArgs extends 
       description = "optional service type")
   public String serviceType = SliderKeys.APP_TYPE;
 
-
   @Parameter(names = {ARG_VERBOSE},
       description = "verbose output")
   public boolean verbose;
@@ -149,11 +148,11 @@ public class ActionRegistryArgs extends 
     return arg ? 1 : 0;
   }
 
-  private String iff(String arg, boolean val) {
+  private String ifdef(String arg, boolean val) {
     return val ? (arg + " "): "";
   }
 
-  private String iff (String arg, String val) {
+  private String ifdef(String arg, String val) {
     if (is(val)) {
       return arg + " " + val + " ";
     } else {
@@ -165,23 +164,24 @@ public class ActionRegistryArgs extends 
   public String toString() {
     final StringBuilder sb =
         new StringBuilder(ACTION_REGISTRY);
-    sb.append(iff(ARG_LIST, list));
-    sb.append(iff(ARG_LISTCONF, listConf));
-    sb.append(iff(ARG_LISTFILES, listFiles));
-    sb.append(iff(ARG_GETCONF, listFiles));
-    sb.append(iff(ARG_GETFILES, listFiles));
+    sb.append(' ');
+    sb.append(ifdef(ARG_LIST, list));
+    sb.append(ifdef(ARG_LISTCONF, listConf));
+    sb.append(ifdef(ARG_LISTFILES, listFiles));
+    sb.append(ifdef(ARG_GETCONF, getConf));
+    sb.append(ifdef(ARG_GETFILES, getFiles));
 
-    sb.append(iff(ARG_NAME, name));
-    sb.append(iff(ARG_SERVICETYPE, serviceType));
+    sb.append(ifdef(ARG_NAME, name));
+    sb.append(ifdef(ARG_SERVICETYPE, serviceType));
 
 
-    sb.append(iff(ARG_VERBOSE, verbose));
-    sb.append(iff(ARG_INTERNAL, internal));
+    sb.append(ifdef(ARG_VERBOSE, verbose));
+    sb.append(ifdef(ARG_INTERNAL, internal));
 
     if (dest != null) {
-      sb.append(iff(ARG_DEST, dest.toString()));
+      sb.append(ifdef(ARG_DEST, dest.toString()));
     }
-    sb.append(iff(ARG_FORMAT, format));
+    sb.append(ifdef(ARG_FORMAT, format));
 
     return sb.toString();
   }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java Thu May 15 12:42:51 2014
@@ -69,7 +69,7 @@ public interface Arguments {
   String ARG_RESOURCE_MANAGER = "--rm";
   String ARG_RESOURCE_OPT = "--resopt";
   String ARG_RESOURCE_OPT_SHORT = "-ro";
-  String ARG_SERVICETYPE = "--servictype";
+  String ARG_SERVICETYPE = "--servicetype";
   String ARG_SYSPROP = "-S";
   String ARG_TEMPLATE = "--template";
   String ARG_VERBOSE = "--verbose";

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/RegisteredEndpoint.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/RegisteredEndpoint.java?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/RegisteredEndpoint.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/RegisteredEndpoint.java Thu May 15 12:42:51 2014
@@ -159,8 +159,30 @@ public class RegisteredEndpoint {
     }
   }
 
-  
-  
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder();
+    if (TYPE_URL.equals(type)) {
+      sb.append(value);
+    } else {
+      sb.append("protocol='").append(protocol).append('\'');
+      sb.append(" value='").append(value).append('\'');
+      sb.append(" type='").append(type).append('\'');
+    }
+    sb.append(" -- \"").append(description).append('"');
+    return sb.toString();
+  }
+
+  public boolean isHttpProtocol() {
+    return PROTOCOL_HTTP.equals(protocol) || PROTOCOL_HTTPS.equals(protocol);
+  }
+
+  /**
+   * Verify that an endpoint is of the desired type
+   * @param desiredType desired type
+   * @throws SliderException if it is not
+   */
   public void verifyEndpointType(String desiredType) throws SliderException {
     if (!type.equals(desiredType)) {
       throw new SliderException(-1, "Body of endpoint is of type %s and not %s",

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/ServiceInstanceData.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/ServiceInstanceData.java?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/ServiceInstanceData.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/ServiceInstanceData.java Thu May 15 12:42:51 2014
@@ -22,6 +22,8 @@ import org.codehaus.jackson.annotate.Jso
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
 
 /**
  * Service instance data to serialize with JSON
@@ -54,6 +56,35 @@ public class ServiceInstanceData impleme
     sb.append('}');
     return sb.toString();
   }
+
+  /**
+   * get the internal or external registry
+   * @param external flag to indicate the external endpoints
+   * @return a view -which may be null
+   */
+  public RegistryView getRegistryView(boolean external) {
+    return external ? externalView : internalView;
+  }
+
+  /**
+   * List the internal or external endpoints. This returns
+   * an empty list if there are none registered
+   * @param external flag to indicate the external endpoints
+   * @return a map of published endpoints
+   */
+  public Map<String, RegisteredEndpoint> listEndpoints(boolean external) {
+    RegistryView view = getRegistryView(external);
+    if (view == null) {
+      return new HashMap<>(0);
+    }
+    Map<String, RegisteredEndpoint> endpoints = view.endpoints;
+    if (endpoints != null) {
+      return endpoints;
+    } else {
+      return new HashMap<>(0);
+    }
+  }
+  
 }
 
 

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java Thu May 15 12:42:51 2014
@@ -122,20 +122,25 @@ public class RegistryRetriever {
 
   /**
    * Get a complete configuration, with all values
+   * @param configSet
    * @param name name of the configuration
    * @param external flag to indicate that it is an external configuration
    * @return the retrieved config
    * @throws IOException IO problems
    */
-  public PublishedConfiguration retrieveConfiguration(String name,
+  public PublishedConfiguration retrieveConfiguration(PublishedConfigSet configSet,
+      String name,
       boolean external) throws IOException {
+    if (!configSet.contains(name)) {
+      throw new FileNotFoundException("Unknown configuration " + name);
+    }
     String confURL = getRegistryView(external).configurationsURL;
     confURL = SliderUtils.appendToURL(confURL, name);
     try {
       WebResource webResource = jsonResource(confURL);
-      log.debug("GET {}", confURL);
-      PublishedConfiguration configSet = webResource.get(PublishedConfiguration.class);
-      return configSet;
+      PublishedConfiguration publishedConf =
+          webResource.get(PublishedConfiguration.class);
+      return publishedConf;
     } catch (UniformInterfaceException e) {
       throw ExceptionConverter.convertJerseyException(confURL, e);
     }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java Thu May 15 12:42:51 2014
@@ -78,12 +78,14 @@ public class CuratorService extends Abst
     }
   }
 
-  public String pathForName(String name) {
-    return ZKPaths.makePath(getBasePath(), name);
+  public String pathForServicetype(String servicetype) {
+    return ZKPaths.makePath(getBasePath(), servicetype);
   }
 
-  protected String pathForInstance(String name, String id) {
-    return ZKPaths.makePath(pathForName(name), id);
+  protected String pathForInstance(String servicetype, String id) {
+    Preconditions.checkNotNull(servicetype);
+    Preconditions.checkNotNull(id);
+    return ZKPaths.makePath(pathForServicetype(servicetype), id);
   }
 
   public String getBasePath() {

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java Thu May 15 12:42:51 2014
@@ -20,15 +20,21 @@ package org.apache.slider.server.service
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import org.apache.commons.lang.StringUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.x.discovery.ServiceDiscovery;
 import org.apache.curator.x.discovery.ServiceInstance;
 import org.apache.curator.x.discovery.ServiceInstanceBuilder;
 import org.apache.curator.x.discovery.ServiceType;
 import org.apache.curator.x.discovery.UriSpec;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.slider.common.params.ActionRegistryArgs;
+import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.exceptions.BadClusterStateException;
+import org.apache.slider.core.exceptions.UnknownApplicationInstanceException;
 import org.apache.slider.core.persist.JsonSerDeser;
-import org.apache.slider.server.services.registry.RegistryViewForProviders;
+import org.apache.slider.core.registry.info.ServiceInstanceData;
+import org.apache.slider.server.services.registry.SliderRegistryService;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -162,12 +168,12 @@ public class RegistryBinderService<Paylo
   }
 
 
-  public List<String> instanceIDs(String servicename) throws Exception {
+  public List<String> instanceIDs(String servicetype) throws Exception {
+    Preconditions.checkNotNull(servicetype);
     List<String> instanceIds;
-
     try {
       instanceIds =
-        getCurator().getChildren().forPath(pathForName(servicename));
+        getCurator().getChildren().forPath(pathForServicetype(servicetype));
     } catch (KeeperException.NoNodeException e) {
       instanceIds = Lists.newArrayList();
     }
@@ -178,14 +184,15 @@ public class RegistryBinderService<Paylo
   /**
    * Return a service instance POJO
    *
-   * @param name name of the service
+   * @param servicetype name of the service
    * @param id ID of the instance
    * @return the instance or <code>null</code> if not found
    * @throws Exception errors
    */
-  public CuratorServiceInstance<Payload> queryForInstance(String name, String id) throws
+  public CuratorServiceInstance<Payload> queryForInstance(String servicetype, String id) throws
                                                                          Exception {
-    String path = pathForInstance(name, id);
+
+    String path = pathForInstance(servicetype, id);
     try {
       byte[] bytes = getCurator().getData().forPath(path);
       return deser.fromBytes(bytes);
@@ -197,19 +204,19 @@ public class RegistryBinderService<Paylo
   
   /**
    * List all the instances
-   * @param name name of the service
+   * @param servicetype name of the service
    * @return a list of instances and their payloads
    * @throws IOException any problem
    */
-  public List<CuratorServiceInstance<Payload>> listInstances(String name) throws
+  public List<CuratorServiceInstance<Payload>> listInstances(String servicetype) throws
     IOException {
     try {
-      List<String> instanceIDs = instanceIDs(name);
+      List<String> instanceIDs = instanceIDs(servicetype);
       List<CuratorServiceInstance<Payload>> instances =
         new ArrayList<>(instanceIDs.size());
       for (String instanceID : instanceIDs) {
         CuratorServiceInstance<Payload> instance =
-          queryForInstance(name, instanceID);
+          queryForInstance(servicetype, instanceID);
         instances.add(instance);
       }
       return instances;
@@ -220,6 +227,54 @@ public class RegistryBinderService<Paylo
     }
   }
 
+  /**
+   * Find an instance with a given ID
+   * @param instances instances
+   * @param name ID to look for
+   * @return
+   */
+  public CuratorServiceInstance<Payload> findByID(List<CuratorServiceInstance<Payload>> instances, String name) {
+    Preconditions.checkNotNull(name);
+    for (CuratorServiceInstance<Payload> instance : instances) {
+      if (instance.id.equals(name)) {
+        return instance;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * List registry entries. If a name was given, then the single match is returned
+   * -otherwise all entries matching the service type
+   * @param serviceType service type
+   * @param name an optional name
+   * @return the (non-empty) list of instances that match the criteria
+   * @throws UnknownApplicationInstanceException if there were no matches
+   * @throws IOException
+   */
+  public List<CuratorServiceInstance<Payload>> findInstances(String serviceType,
+      String name)
+      throws UnknownApplicationInstanceException, IOException {
+    List<CuratorServiceInstance<Payload>> instances =
+        listInstances(serviceType);
+    if (instances.isEmpty()) {
+      throw new UnknownApplicationInstanceException(
+          "No registry entries for service type %s",
+          serviceType);
+    }
+    if (StringUtils.isNotEmpty(name)) {
+      CuratorServiceInstance<Payload> foundInstance = findByID(instances, name);
+      if (foundInstance == null) {
+        throw new UnknownApplicationInstanceException(
+            "No registry entries for service name %s of service type %s",
+            name,
+            serviceType);
+      }
+      instances.clear();
+      instances.add(foundInstance);
+    }
+    return instances;
+  }
 
   public Collection<String> queryForNames() throws IOException {
     try {

Modified: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy Thu May 15 12:42:51 2014
@@ -28,6 +28,7 @@ import org.apache.slider.api.ClusterNode
 import org.apache.slider.client.SliderClient
 import org.apache.slider.common.SliderKeys
 import org.apache.slider.common.params.ActionRegistryArgs
+import org.apache.slider.core.exceptions.UnknownApplicationInstanceException
 import org.apache.slider.core.main.ServiceLauncher
 import org.apache.slider.core.persist.JsonSerDeser
 import org.apache.slider.core.registry.docstore.PublishedConfigSet
@@ -96,23 +97,23 @@ class TestStandaloneRegistryAM extends A
 
 
     String username = client.username
-    def serviceRegistryClient = client.YARNRegistryClient
+    def yarnRegistryClient = client.YARNRegistryClient
     describe("list of all applications")
     logApplications(apps)
     describe("apps of user $username")
-    List<ApplicationReport> userInstances = serviceRegistryClient.listInstances()
+    List<ApplicationReport> userInstances = yarnRegistryClient.listInstances()
     logApplications(userInstances)
     assert userInstances.size() == 1
     describe("named app $clustername")
-    ApplicationReport instance = serviceRegistryClient.findInstance(clustername)
+    ApplicationReport instance = yarnRegistryClient.findInstance(clustername)
     logReport(instance)
     assert instance != null
 
     //switch to the ZK-based registry
 
     describe "service registry names"
-    SliderRegistryService registry = client.registry
-    def names = registry.queryForNames();
+    SliderRegistryService registryService = client.registry
+    def names = registryService.queryForNames();
     dumpRegistryNames(names)
 
     List<String> instanceIds = client.listRegistryInstanceIDs()
@@ -121,8 +122,8 @@ class TestStandaloneRegistryAM extends A
     dumpRegistryInstanceIDs(instanceIds)
     assert instanceIds.size() == 1
 
-    List<CuratorServiceInstance<ServiceInstanceData>> instances = client.listRegistryInstances(
-    )
+    List<CuratorServiceInstance<ServiceInstanceData>> instances =
+        client.listRegistryInstances()
     dumpRegistryInstances(instances)
 
     assert instances.size() == 1
@@ -147,8 +148,8 @@ class TestStandaloneRegistryAM extends A
     describe("Publisher")
 
     def publishedJSON = GET(publisherURL)
-    log.info(publishedJSON)
-    JsonSerDeser< PublishedConfigSet> serDeser= new JsonSerDeser<PublishedConfigSet>(
+//    log.info(publishedJSON)
+    JsonSerDeser< PublishedConfigSet> serDeser= new JsonSerDeser<>(
         PublishedConfigSet)
     def configSet = serDeser.fromJson(publishedJSON)
     assert configSet.size() >= 1
@@ -161,7 +162,7 @@ class TestStandaloneRegistryAM extends A
     def yarnSitePublisher = appendToURL(publisher, ARTIFACT_NAME)
 
     String confJSON = GET(yarnSitePublisher)
-    log.info(confJSON)
+//    log.info(confJSON)
     JsonSerDeser< PublishedConfiguration> confSerDeser =
         new JsonSerDeser<PublishedConfiguration>(PublishedConfiguration)
 
@@ -194,22 +195,25 @@ class TestStandaloneRegistryAM extends A
     log.info(GET(registryURL, RestPaths.REGISTRY_SERVICE ))
 
 
-    describe "Registry Retrieval"
+    describe "Registry Retrieval Class"
     // retrieval
 
     RegistryRetriever retriever = new RegistryRetriever(serviceInstanceData)
     log.info retriever.toString()
     
     assert retriever.hasConfigurations(true)
-    def externalConf = retriever.getConfigurations(true)
-    externalConf.keys().each { String key ->
-      def config = externalConf.get(key)
+    PublishedConfigSet externalConfSet = retriever.getConfigurations(true)
+    externalConfSet.keys().each { String key ->
+      def config = externalConfSet.get(key)
       log.info "$key -- ${config.description}"
     }
-    assert externalConf[ARTIFACT_NAME]
+    assert externalConfSet[ARTIFACT_NAME]
 
 
-    def yarnSite = retriever.retrieveConfiguration(ARTIFACT_NAME, true)
+    def yarnSite = retriever.retrieveConfiguration(
+        externalConfSet,
+        ARTIFACT_NAME,
+        true)
     assert !yarnSite.empty
     def siteXML = yarnSite.asConfiguration()
     def rmHostnameViaClientSideXML = parsedProps.get(
@@ -220,32 +224,69 @@ class TestStandaloneRegistryAM extends A
   /* TODO SLIDER-52 PublishedConfiguration XML conf values are not resolved until client-side
    assert rmAddrViaClientSideXML == rmAddrFromDownloadedProperties
   */  
+    describe "fetch missing artifact"
+    try {
+      retriever.retrieveConfiguration(externalConfSet, "no-such-artifact", true)
+      fail("expected a failure")
+    } catch (FileNotFoundException expected) {
+      // expected
+    }
     describe "Internal configurations"
     assert !retriever.hasConfigurations(false)
     try {
       retriever.getConfigurations(false)
-      fail( "expected a failure")
+      fail("expected a failure")
     } catch (FileNotFoundException expected) {
-      //expected
+      // expected
     }
 
 
     // retrieval via API
     ActionRegistryArgs registryArgs = new ActionRegistryArgs()
-    registryArgs.name = serviceInstanceData.id;
     registryArgs.verbose = true
 
-    // list
+    // list all
     registryArgs.list = true;
     describe registryArgs.toString()
-    assert 0 == client.actionRegistry(registryArgs)
+    client.actionRegistry(registryArgs)
+
+    // list a named instance and expect a  failure
+    registryArgs.list = true;
+    registryArgs.name = "unknown"
+    try {
+      client.actionRegistryList(registryArgs)
+    } catch (UnknownApplicationInstanceException ignored) {
+      // expected 
+    }
+
+    // list all instances of an alternate type and expect failure
+    registryArgs.list = true;
+    registryArgs.name = null
+    registryArgs.serviceType = "org.apache.hadoop"
+    try {
+      client.actionRegistryList(registryArgs)
+    } catch (UnknownApplicationInstanceException ignored) {
+      // expected 
+    }
+
+    //set the name
+    registryArgs.name = serviceInstanceData.id;
+    registryArgs.serviceType = SliderKeys.APP_TYPE
+    
+
+    //now expect list to work
+    describe registryArgs.toString()
+
+    def listedInstance = client.actionRegistryList(registryArgs)
+    assert listedInstance[0].id == serviceInstanceData.id
+    
 
     // listconf 
     registryArgs.list = false;
     registryArgs.listConf = true
     describe registryArgs.toString() 
     
-    assert 0 == client.actionRegistry(registryArgs)
+    client.actionRegistry(registryArgs)
 
     // listconf --internal
     registryArgs.list = false;
@@ -253,7 +294,7 @@ class TestStandaloneRegistryAM extends A
     registryArgs.internal = true
     describe registryArgs.toString()
     try {
-      assert 0 == client.actionRegistry(registryArgs)
+      client.actionRegistry(registryArgs)
       fail("expected a failure")
     } catch (FileNotFoundException expected) {
       //expected
@@ -263,30 +304,36 @@ class TestStandaloneRegistryAM extends A
     registryArgs.listConf = false
     registryArgs.internal = false
     registryArgs.format = "properties"
-    registryArgs.getConf = ARTIFACT_NAME
+
+    def yarn_site_config = PublishedArtifacts.YARN_SITE_CONFIG
+    registryArgs.getConf = yarn_site_config
+    
     
     describe registryArgs.toString()
-    assert 0 == client.actionRegistry(registryArgs)
+    client.actionRegistry(registryArgs)
 
     File outputDir = new File("target/test_standalone_registry_am/output")
     outputDir.mkdirs()
 
     registryArgs.dest = outputDir
     describe registryArgs.toString()
-    assert 0 == client.actionRegistry(registryArgs)
-    assert new File(outputDir,ARTIFACT_NAME + ".properties").exists()
+    client.actionRegistry(registryArgs)
+    assert new File(outputDir, yarn_site_config + ".properties").exists()
 
     registryArgs.format = "xml"
-    assert 0 == client.actionRegistry(registryArgs)
-    assert new File(outputDir,ARTIFACT_NAME + ".xml").exists()
+    client.actionRegistry(registryArgs)
+    assert new File(outputDir, yarn_site_config + ".xml").exists()
 
+    describe registryArgs.toString()
+    registryArgs.getConf = "undefined-file"
+    client.actionRegistry(registryArgs)
 
 
     describe "freeze cluster"
     //now kill that cluster
     assert 0 == clusterActionFreeze(client, clustername)
     //list it & See if it is still there
-    ApplicationReport oldInstance = serviceRegistryClient.findInstance(
+    ApplicationReport oldInstance = yarnRegistryClient.findInstance(
         clustername)
     assert oldInstance != null
     assert oldInstance.yarnApplicationState >= YarnApplicationState.FINISHED

Modified: incubator/slider/trunk/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy (original)
+++ incubator/slider/trunk/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/CommandTestBase.groovy Thu May 15 12:42:51 2014
@@ -150,7 +150,7 @@ abstract class CommandTestBase extends S
    * @param commands
    * @return the shell
    */
-  public static SliderShell slider(List<String> commands) {
+  public static SliderShell slider(Collection<String> commands) {
     SliderShell shell = new SliderShell(commands)
     shell.execute()
     return shell
@@ -162,7 +162,7 @@ abstract class CommandTestBase extends S
    * @param commands commands
    * @return
    */
-  public static SliderShell slider(int exitCode, List<String> commands) {
+  public static SliderShell slider(int exitCode, Collection<String> commands) {
     return SliderShell.run(commands, exitCode)
   }
 
@@ -197,7 +197,7 @@ abstract class CommandTestBase extends S
         ACTION_EXISTS, name
     ]
     if (live) {
-      args << Arguments.ARG_LIVE
+      args << ARG_LIVE
     }
     slider(args)
   }
@@ -217,6 +217,14 @@ abstract class CommandTestBase extends S
         ACTION_FREEZE, name
     ])
   }
+  
+  static SliderShell freeze(String name, Collection<String> args) {
+    slider([ACTION_FREEZE, name] + args)
+  }
+
+  static SliderShell freezeForce(String name) {
+    freeze(name, [ARG_FORCE])
+  }
 
   static SliderShell getConf(String name) {
     slider([
@@ -230,7 +238,7 @@ abstract class CommandTestBase extends S
              ACTION_GETCONF, name
          ])
   }
-
+  
   static SliderShell killContainer(String name, String containerID) {
     slider(0,
          [
@@ -239,12 +247,6 @@ abstract class CommandTestBase extends S
              containerID
          ])
   }
-  
-  static SliderShell freezeForce(String name) {
-    slider([
-        ACTION_FREEZE, ARG_FORCE, name
-    ])
-  }
 
   static SliderShell list(String name) {
     List<String> cmd = [
@@ -292,6 +294,21 @@ abstract class CommandTestBase extends S
          ])
   }
 
+  static SliderShell thaw(String name, Collection<String> args) {
+    slider([ACTION_THAW, name] + args)
+  }
+  
+  static SliderShell registry(int result, Collection<String> commands) {
+    slider(result,
+         [ ACTION_REGISTRY ] + commands
+    )
+  }
+  static SliderShell registry(Collection<String> commands) {
+    slider(0,
+         [ ACTION_REGISTRY ] + commands
+    )
+  }
+
   /**
    * Ensure that a cluster has been destroyed
    * @param name

Modified: incubator/slider/trunk/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy (original)
+++ incubator/slider/trunk/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/SliderShell.groovy Thu May 15 12:42:51 2014
@@ -46,7 +46,7 @@ class SliderShell extends Shell {
    * Build the command
    * @param commands
    */
-  SliderShell(List<String> commands) {
+  SliderShell(Collection<String> commands) {
     super(BASH)
     assert confDir != null;
     assert script != null;
@@ -63,7 +63,7 @@ class SliderShell extends Shell {
     List<String> commandLine = [
         confDirCmd,
     ]
-    if (!slider_classpath_extra.isEmpty()) {
+    if (!slider_classpath_extra.empty) {
       commandLine << env(FuntestProperties.ENV_SLIDER_CLASSPATH_EXTRA,
           SliderUtils.join(slider_classpath_extra, ":", false))
     }
@@ -98,7 +98,7 @@ class SliderShell extends Shell {
    * @param commands
    * @return the shell
    */
-  public static SliderShell run(List<String> commands, int exitCode) {
+  public static SliderShell run(Collection<String> commands, int exitCode) {
     SliderShell shell = new SliderShell(commands)
     shell.execute(exitCode);
     return shell

Modified: incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterLifecycle.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterLifecycle.groovy?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterLifecycle.groovy (original)
+++ incubator/slider/trunk/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterLifecycle.groovy Thu May 15 12:42:51 2014
@@ -114,16 +114,15 @@ public class TestClusterLifecycle extend
 
       //get a slider client against the cluster
       SliderClient sliderClient = bondToCluster(SLIDER_CONFIG, CLUSTER)
-      ClusterDescription cd2 = sliderClient.getClusterDescription()
+      ClusterDescription cd2 = sliderClient.clusterDescription
       assert CLUSTER == cd2.name
 
       log.info("Connected via Client {}", sliderClient.toString())
 
       //freeze
-      slider(0, [
-          SliderActions.ACTION_FREEZE, CLUSTER,
+      freeze(CLUSTER, [
           ARG_WAIT, Integer.toString(FREEZE_WAIT_TIME),
-          ARG_MESSAGE, "freeze-in-testHBaseCreateCluster"
+          ARG_MESSAGE, "freeze-in-test cluster lifecycle"
       ])
 
       //cluster exists if you don't want it to be live
@@ -134,18 +133,17 @@ public class TestClusterLifecycle extend
 
       // thaw then freeze the cluster
 
-      slider(0,
+      thaw(CLUSTER,
            [
-               SliderActions.ACTION_THAW, CLUSTER,
                ARG_WAIT, Integer.toString(THAW_WAIT_TIME),
            ])
       exists(0, CLUSTER)
-      slider(0, [
-          SliderActions.ACTION_FREEZE, CLUSTER,
-          ARG_FORCE,
-          ARG_WAIT, Integer.toString(FREEZE_WAIT_TIME),
-          ARG_MESSAGE, "forced-freeze-in-test"
-      ])
+      freeze(CLUSTER,
+          [
+              ARG_FORCE,
+              ARG_WAIT, Integer.toString(FREEZE_WAIT_TIME),
+              ARG_MESSAGE, "forced-freeze-in-test"
+          ])
 
       //cluster is no longer live
       exists(0, CLUSTER, false)
@@ -156,15 +154,12 @@ public class TestClusterLifecycle extend
       // thaw with a restart count set to enable restart
 
       describe "the kill/restart phase may fail if yarn.resourcemanager.am.max-attempts is too low"
-      slider(0,
+      thaw(CLUSTER,
            [
-               SliderActions.ACTION_THAW, CLUSTER,
                ARG_WAIT, Integer.toString(THAW_WAIT_TIME),
                ARG_DEFINE, SliderXmlConfKeys.KEY_AM_RESTART_LIMIT + "=3"
            ])
 
-
-
       ClusterDescription status = killAmAndWaitForRestart(sliderClient, CLUSTER)
 
       def restarted = status.getInfo(

Modified: incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestFunctionalHBaseCluster.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestFunctionalHBaseCluster.groovy?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestFunctionalHBaseCluster.groovy (original)
+++ incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestFunctionalHBaseCluster.groovy Thu May 15 12:42:51 2014
@@ -22,6 +22,7 @@ import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.conf.Configuration
 import org.apache.slider.common.SliderExitCodes
+import org.apache.slider.common.SliderKeys
 import org.apache.slider.common.SliderXmlConfKeys
 import org.apache.slider.api.ClusterDescription
 import org.apache.slider.api.RoleKeys
@@ -84,8 +85,7 @@ public class TestFunctionalHBaseCluster 
 
     describe description
 
-    int numWorkers = SLIDER_CONFIG.getInt(KEY_SLIDER_TEST_NUM_WORKERS,
-        DEFAULT_SLIDER_NUM_WORKERS);
+    int numWorkers = desiredWorkerCount;
 
     def clusterpath = buildClusterPath(clusterName)
     assert !clusterFS.exists(clusterpath)
@@ -107,7 +107,7 @@ public class TestFunctionalHBaseCluster 
 
     //get a slider client against the cluster
     SliderClient sliderClient = bondToCluster(SLIDER_CONFIG, clusterName)
-    ClusterDescription cd2 = sliderClient.getClusterDescription()
+    ClusterDescription cd2 = sliderClient.clusterDescription
     assert clusterName == cd2.name
 
     log.info("Connected via Client {} with {} workers", sliderClient.toString(),
@@ -121,7 +121,22 @@ public class TestFunctionalHBaseCluster 
     HBaseTestUtils.waitForHBaseRegionServerCount(sliderClient, clusterName,
                                   numWorkers, HBASE_LAUNCH_WAIT_TIME)
 
-    clusterLoadOperations(clusterName, clientConf, numWorkers, roleMap, cd2)
+    clusterOperations(
+        clusterName,
+        sliderClient,
+        clientConf,
+        numWorkers,
+        roleMap,
+        cd2)
+  }
+
+  /**
+   * Override to change policy of the deired no of workers
+   * @return
+   */
+  def int getDesiredWorkerCount() {
+    return SLIDER_CONFIG.getInt(KEY_SLIDER_TEST_NUM_WORKERS,
+        DEFAULT_SLIDER_NUM_WORKERS)
   }
 
 
@@ -130,18 +145,31 @@ public class TestFunctionalHBaseCluster 
   }
 
   /**
-   * Override point for any cluster load operations
-   * @param clientConf
-   * @param numWorkers
+   * Override point for any cluster operations
+   * @param clustername name of cluster
+   * @param sliderClient bonded low level client
+   * @param clientConf config
+   * @param numWorkers no. of workers created
+   * @param roleMap role map
+   * @param cd current cluster
    */
-  public void clusterLoadOperations(
+  public void clusterOperations(
       String clustername,
+      SliderClient sliderClient,
       Configuration clientConf,
       int numWorkers,
       Map<String, Integer> roleMap,
       ClusterDescription cd) {
 
     log.info("Client Configuration = " + ConfigHelper.dumpConfigToString(clientConf))
+    
+    //grab some registry bits
+    registry([ARG_LIST])
+    registry([ARG_LIST, ARG_SERVICETYPE, SliderKeys.APP_TYPE ])
+    registry([ARG_LIST, ARG_SERVICETYPE, ""])
+    registry([ARG_LIST, ARG_SERVICETYPE, ""])
+    
+    
   }
 
 }

Copied: incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseClusterBuildDestroy.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestClusterBuildDestroy.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseClusterBuildDestroy.groovy?p2=incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseClusterBuildDestroy.groovy&p1=incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestClusterBuildDestroy.groovy&r1=1594525&r2=1594902&rev=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestClusterBuildDestroy.groovy (original)
+++ incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseClusterBuildDestroy.groovy Thu May 15 12:42:51 2014
@@ -35,7 +35,7 @@ import org.junit.Test
 
 @CompileStatic
 @Slf4j
-public class TestClusterBuildDestroy extends HBaseCommandTestBase
+public class TestHBaseClusterBuildDestroy extends HBaseCommandTestBase
     implements FuntestProperties, Arguments {
 
 

Copied: incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseClusterLifecycle.groovy (from r1594525, incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestClusterLifecycle.groovy)
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseClusterLifecycle.groovy?p2=incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseClusterLifecycle.groovy&p1=incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestClusterLifecycle.groovy&r1=1594525&r2=1594902&rev=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestClusterLifecycle.groovy (original)
+++ incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseClusterLifecycle.groovy Thu May 15 12:42:51 2014
@@ -34,7 +34,7 @@ import org.junit.Test
 
 @CompileStatic
 @Slf4j
-public class TestClusterLifecycle extends HBaseCommandTestBase
+public class TestHBaseClusterLifecycle extends HBaseCommandTestBase
     implements FuntestProperties, Arguments, SliderExitCodes {
 
 
@@ -112,16 +112,15 @@ public class TestClusterLifecycle extend
 
       //get a slider client against the cluster
       SliderClient sliderClient = bondToCluster(SLIDER_CONFIG, CLUSTER)
-      ClusterDescription cd2 = sliderClient.getClusterDescription()
+      ClusterDescription cd2 = sliderClient.clusterDescription
       assert CLUSTER == cd2.name
 
       log.info("Connected via Client {}", sliderClient.toString())
 
       //freeze
-      slider(0, [
-          SliderActions.ACTION_FREEZE, CLUSTER,
+      freeze(CLUSTER, [
           ARG_WAIT, Integer.toString(FREEZE_WAIT_TIME),
-          ARG_MESSAGE, "freeze-in-testHBaseCreateCluster"
+          ARG_MESSAGE, "freeze-in-test cluster lifecycle"
       ])
 
       //cluster exists if you don't want it to be live
@@ -132,18 +131,17 @@ public class TestClusterLifecycle extend
 
       // thaw then freeze the cluster
 
-      slider(0,
-           [
-               SliderActions.ACTION_THAW, CLUSTER,
-               ARG_WAIT, Integer.toString(THAW_WAIT_TIME),
-           ])
+      thaw(CLUSTER,
+          [
+              ARG_WAIT, Integer.toString(THAW_WAIT_TIME),
+          ])
       exists(0, CLUSTER)
-      slider(0, [
-          SliderActions.ACTION_FREEZE, CLUSTER,
-          ARG_FORCE,
-          ARG_WAIT, Integer.toString(FREEZE_WAIT_TIME),
-          ARG_MESSAGE, "forced-freeze-in-test"
-      ])
+      freeze(CLUSTER,
+          [
+              ARG_FORCE,
+              ARG_WAIT, Integer.toString(FREEZE_WAIT_TIME),
+              ARG_MESSAGE, "forced-freeze-in-test"
+          ])
 
       //cluster is no longer live
       exists(0, CLUSTER, false)
@@ -154,13 +152,11 @@ public class TestClusterLifecycle extend
       // thaw with a restart count set to enable restart
 
       describe "the kill/restart phase may fail if yarn.resourcemanager.am.max-attempts is too low"
-      slider(0,
-           [
-               SliderActions.ACTION_THAW, CLUSTER,
-               ARG_WAIT, Integer.toString(THAW_WAIT_TIME),
-               ARG_DEFINE, SliderXmlConfKeys.KEY_AM_RESTART_LIMIT + "=3"
-           ])
-
+      thaw(CLUSTER,
+          [
+              ARG_WAIT, Integer.toString(THAW_WAIT_TIME),
+              ARG_DEFINE, SliderXmlConfKeys.KEY_AM_RESTART_LIMIT + "=3"
+          ])
 
 
       ClusterDescription status = killAmAndWaitForRestart(sliderClient, CLUSTER)

Modified: incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseIntegration.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseIntegration.groovy?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseIntegration.groovy (original)
+++ incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseIntegration.groovy Thu May 15 12:42:51 2014
@@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.Integrati
 import org.apache.hadoop.hbase.IntegrationTestingUtility
 import org.apache.hadoop.util.ToolRunner
 import org.apache.slider.api.ClusterDescription
+import org.apache.slider.client.SliderClient
 import org.apache.slider.providers.hbase.HBaseConfigFileOptions;
 
 /* Runs IntegrationTestIngest on cluster
@@ -38,8 +39,9 @@ class TestHBaseIntegration extends TestF
   }
 
   @Override
-  void clusterLoadOperations(
+  void clusterOperations(
       String clustername,
+      SliderClient sliderClient,
       Configuration clientConf,
       int numWorkers,
       Map<String, Integer> roleMap,

Modified: incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseLoad.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseLoad.groovy?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseLoad.groovy (original)
+++ incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseLoad.groovy Thu May 15 12:42:51 2014
@@ -21,6 +21,7 @@ package org.apache.slider.providers.hbas
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.hbase.util.LoadTestTool
 import org.apache.slider.api.ClusterDescription
+import org.apache.slider.client.SliderClient
 import org.apache.slider.providers.hbase.HBaseConfigFileOptions
 
 class TestHBaseLoad extends TestFunctionalHBaseCluster {
@@ -31,8 +32,9 @@ class TestHBaseLoad extends TestFunction
   }
 
   @Override
-  void clusterLoadOperations(
+  void clusterOperations(
       String clustername,
+      SliderClient sliderClient,
       Configuration clientConf,
       int numWorkers,
       Map<String, Integer> roleMap,

Modified: incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseNodeFailure.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseNodeFailure.groovy?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseNodeFailure.groovy (original)
+++ incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestHBaseNodeFailure.groovy Thu May 15 12:42:51 2014
@@ -43,13 +43,13 @@ class TestHBaseNodeFailure extends TestF
   }
 
   @Override
-  void clusterLoadOperations(
+  void clusterOperations(
       String clustername,
+      SliderClient sliderClient,
       Configuration clientConf,
       int numWorkers,
       Map<String, Integer> roleMap,
       ClusterDescription cd) {
-    SliderClient sliderClient = bondToCluster(SLIDER_CONFIG, clusterName)
 
 
     killInstanceOfRole(sliderClient, HBaseKeys.ROLE_WORKER)

Modified: incubator/slider/trunk/src/site/markdown/manpage.md
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/src/site/markdown/manpage.md?rev=1594902&r1=1594901&r2=1594902&view=diff
==============================================================================
--- incubator/slider/trunk/src/site/markdown/manpage.md (original)
+++ incubator/slider/trunk/src/site/markdown/manpage.md Thu May 15 12:42:51 2014
@@ -343,6 +343,53 @@ Example
     slider list
     slider list instance1
 
+### `registry (--list | --listconf | --getconf <conf>) [--name <name>] [--servicetype <servicetype>] [--verbose]`
+
+List registered application instances visible to the user.
+
+The `--name <name>` option names the registry entry to work with. For slider applications,
+this is the application instance
+
+
+The `--servicetype <servicetype>` option allows a different service type to be chosen.
+The default is `"org.apache.slider`
+
+The `--verbose` flag triggers more verbose output on the operations
+
+The `--internal` flag indicates the configurations to be listed and retrieved
+are from the "internal" list of configuration data provided for use within a
+deployed application. 
+
+Operations:
+
+#### `registry --list  [--servicetype <servicetype>] [--name <name>] [--verbose]`
+
+List all services of the service type and optionally the name. 
+ 
+If there were no matches then the operation fails with the `EXIT_UNKNOWN_INSTANCE` code
+
+ 
+ 
+ 
+#### `registry --listconf [--name <name>]  [--internal] [--servicetype <servicetype>]`
+
+List the configurations exported by of a named application
+
+
+
+#### `registry --getconf <configuration> [--format (xml|json|properties)] [--dest <path>] [--internal] ` get the configuration
+ 
+Get a named configuration
+
+  `--dest <path>` : the filename or directory to save a configuration to.
+  `--format (xml|json|properties)` defines the output format
+  
+ 
+ 
+ 
+
+
+
 ### `status <name> [--out <filename>]`
 
 Get the status of the named application instance in JSON format. A filename can be used to