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/10/09 00:53:13 UTC

[24/50] [abbrv] git commit: SLIDER-149 migration off curator-x-discovery: read only REST API (work in progress)

SLIDER-149 migration off curator-x-discovery: read only REST API (work in progress)


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/44e61659
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/44e61659
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/44e61659

Branch: refs/heads/develop
Commit: 44e61659f0c9cefb16b2cbc1f63447ee4c96c4fc
Parents: 8c7f6b2
Author: Steve Loughran <st...@apache.org>
Authored: Thu Sep 25 10:50:08 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Sep 25 10:50:08 2014 +0100

----------------------------------------------------------------------
 .../funtest/accumulo/AccumuloBasicIT.groovy     |   8 +-
 .../org/apache/slider/client/SliderClient.java  | 271 ++++---------------
 .../slider/client/SliderClusterOperations.java  |   1 -
 .../slider/core/launch/AbstractLauncher.java    |   8 +-
 .../core/launch/JavaCommandLineBuilder.java     |   4 +-
 .../registry/info/CustomRegistryConstants.java  |   1 +
 .../core/registry/info/ServiceInstanceData.java |   1 +
 .../registry/retrieve/RegistryRetriever.java    |  10 -
 .../providers/AbstractProviderService.java      |  46 ++--
 .../slider/providers/ProviderService.java       |  11 +-
 .../providers/agent/AgentProviderService.java   |  22 +-
 .../slideram/SliderAMProviderService.java       |  37 +--
 .../server/appmaster/SliderAppMaster.java       |  82 ++----
 .../server/appmaster/web/SliderAMWebApp.java    |  22 +-
 .../slider/server/appmaster/web/WebAppApi.java  |   4 +
 .../server/appmaster/web/WebAppApiImpl.java     |  17 +-
 .../appmaster/web/rest/AMWebServices.java       |   9 +
 .../server/appmaster/web/rest/RestPaths.java    |   4 +-
 .../web/rest/registry/RegistryResource.java     | 130 +++++++++
 .../server/services/curator/CuratorHelper.java  |  11 -
 .../server/services/curator/CuratorService.java |   1 +
 .../curator/CuratorServiceInstance.java         |   1 +
 .../curator/CuratorServiceInstances.java        |   1 +
 .../server/services/curator/CuratorUriSpec.java |   1 +
 .../services/curator/RegistryBinderService.java |   1 +
 .../registry/RegistryDiscoveryContext.java      |   1 +
 .../registry/RegistryRestResources.java         |   1 +
 .../registry/RegistryServiceConstants.java      |   1 +
 .../registry/RegistryViewForProviders.java      |  48 ----
 .../registry/SliderRegistryService.java         |   8 +-
 .../utility/AbstractSliderLaunchedService.java  |  52 +---
 .../YarnRegistryViewForProviders.java           |  32 ++-
 .../standalone/TestStandaloneAgentAM.groovy     |  13 -
 .../TestStandaloneYarnRegistryAM.groovy         |  46 ++--
 .../model/mock/MockProviderService.groovy       |   4 -
 .../view/TestClusterSpecificationBlock.groovy   |   7 +-
 .../web/view/TestContainerStatsBlock.groovy     |   7 +-
 .../appmaster/web/view/TestIndexBlock.groovy    |   8 +-
 .../apache/slider/test/SliderTestUtils.groovy   |  13 +-
 .../web/rest/agent/TestAMAgentWebServices.java  |   3 +-
 .../management/TestAMManagementWebServices.java |   2 +-
 .../publisher/TestAgentProviderService.java     |   5 +-
 .../minicluster/live/TestHBaseMaster.groovy     |  33 +--
 .../minicluster/live/TestTwoLiveClusters.groovy |  35 ++-
 44 files changed, 409 insertions(+), 614 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy
index 475c4e3..613769e 100644
--- a/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy
+++ b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy
@@ -25,6 +25,7 @@ import org.apache.hadoop.security.ProviderUtils
 import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.security.alias.CredentialProvider
 import org.apache.hadoop.security.alias.CredentialProviderFactory
+import org.apache.hadoop.yarn.registry.client.types.ServiceRecord
 import org.apache.slider.accumulo.CustomAuthenticator
 import org.apache.slider.api.ClusterDescription
 import org.apache.slider.client.SliderClient
@@ -170,10 +171,9 @@ class AccumuloBasicIT extends AccumuloAgentCommandTestBase {
     int tries = 5
     while (true) {
       try {
-        CuratorServiceInstance<ServiceInstanceData> instance =
-          sliderClient.getRegistry().queryForInstance(SliderKeys.APP_TYPE, clusterName)
-        ServiceInstanceData serviceInstanceData = instance.payload
-        RegistryRetriever retriever = new RegistryRetriever(serviceInstanceData)
+        ServiceRecord instance =
+          sliderClient.lookupServiceRecord(SliderKeys.APP_TYPE, clusterName)
+        RegistryRetriever retriever = new RegistryRetriever(instance)
         PublishedConfiguration configuration = retriever.retrieveConfiguration(
           retriever.getConfigurations(true), "quicklinks", true)
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index 1afd161..e68a7cb 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -40,10 +40,10 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
-import org.apache.hadoop.yarn.registry.client.binding.BindingUtils;
-import org.apache.hadoop.yarn.registry.client.binding.RecordOperations;
-import org.apache.hadoop.yarn.registry.client.binding.ZKPathDumper;
-import org.apache.hadoop.yarn.registry.client.services.RegistryOperationsService;
+import org.apache.hadoop.yarn.registry.client.api.RegistryOperations;
+import static org.apache.hadoop.yarn.registry.client.binding.RegistryOperationUtils.*;
+
+import org.apache.hadoop.yarn.registry.client.binding.RegistryOperationUtils;
 import org.apache.hadoop.yarn.registry.client.types.Endpoint;
 import org.apache.hadoop.yarn.registry.client.types.RegistryPathStatus;
 import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
@@ -120,8 +120,6 @@ import org.apache.slider.providers.agent.AgentKeys;
 import org.apache.slider.providers.slideram.SliderAMClientProvider;
 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.registry.SliderRegistryService;
 import org.apache.slider.server.services.utility.AbstractSliderLaunchedService;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -176,13 +174,13 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   private SliderYarnClientImpl yarnClient;
   private YarnAppListClient YarnAppListClient;
   private AggregateConf launchedInstanceDefinition;
-  private SliderRegistryService registry;
+//  private SliderRegistryService registry;
 
 
   /**
    * The YARN registry service
    */
-  private RegistryOperationsService registryOperations;
+  private RegistryOperations registryOperations;
 
   /**
    * Constructor
@@ -394,7 +392,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
 
   /**
    * Delete the zookeeper node associated with the calling user and the cluster
+   * TODO: YARN registry operations
    **/
+  @Deprecated
   @VisibleForTesting
   public boolean deleteZookeeperNode(String clusterName) throws YarnException, IOException {
     String user = getUsername();
@@ -1454,7 +1454,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   }
 
   public String getUsername() throws IOException {
-    return UserGroupInformation.getCurrentUser().getShortUserName();
+    return RegistryOperationUtils.currentUser();
   }
 
   /**
@@ -2407,58 +2407,32 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    * @throws IOException Network or other problems
    */
   @VisibleForTesting
-  @Deprecated
-  public List<ServiceInstanceData> actionRegistryList(
-      ActionRegistryArgs registryArgs)
-      throws YarnException, IOException {
-    SliderRegistryService registryService = getRegistry();
-    String serviceType = registryArgs.serviceType;
-    String name = registryArgs.name;
-    List<CuratorServiceInstance<ServiceInstanceData>> instances =
-        registryService.findInstances(serviceType, name);
-    int size = instances.size();
-    if (size == 0) {
-      throw new FileNotFoundException("No entries for servicetype "
-                                      + serviceType
-                                      + " name " + name);
-    }
-    List<ServiceInstanceData> sids = new ArrayList<ServiceInstanceData>(size);
-    for (CuratorServiceInstance<ServiceInstanceData> instance : instances) {
-      ServiceInstanceData payload = instance.payload;
-      logInstance(payload, registryArgs.verbose);
-      sids.add(payload);
-    }
-    return sids;
-  }
-
-  /**
-   * Registry operation
-   *
-   * @param registryArgs registry Arguments
-   * @return the instances (for tests)
-   * @throws YarnException YARN problems
-   * @throws IOException Network or other problems
-   */
-  @VisibleForTesting
   public Collection<ServiceRecord> actionRegistryListYarn(
       ActionRegistryArgs registryArgs)
       throws YarnException, IOException {
     String serviceType = registryArgs.serviceType;
     String name = registryArgs.name;
-    RegistryOperationsService operations = getRegistryOperations();
+    RegistryOperations operations = getRegistryOperations();
     Collection<ServiceRecord> serviceRecords;
     if (StringUtils.isEmpty(name)) {
       String serviceclassPath =
-          BindingUtils.serviceclassPath(BindingUtils.currentUser(),
+          serviceclassPath(
+              currentUser(),
               serviceType);
-      RegistryPathStatus[] listDir;
-      listDir = operations.list(serviceclassPath);
-      if (listDir.length == 0) {
-        throw new PathNotFoundException("records under "
-                                        + serviceclassPath);
+
+      try {
+        Map<String, ServiceRecord> recordMap =
+            listServiceRecords(operations, serviceclassPath);
+        RegistryPathStatus[] listDir;
+        if (recordMap.isEmpty()) {
+          throw new UnknownApplicationInstanceException(
+              "No applications registered under " + serviceclassPath);
+        }
+        serviceRecords = recordMap.values();
+      } catch (PathNotFoundException e) {
+        throw new UnknownApplicationInstanceException(e.getPath().toString(),
+            e);
       }
-      serviceRecords =
-          RecordOperations.extractServiceRecords(operations, listDir).values();
     } else {
       ServiceRecord instance = lookupServiceRecord(registryArgs);
       serviceRecords = new ArrayList<ServiceRecord>(1);
@@ -2517,33 +2491,6 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     }
   }
 
-  /**
-   * list configs available for an instance
-   *
-   * @param registryArgs registry Arguments
-   * @throws YarnException YARN problems
-   * @throws IOException Network or other problems
-   */
-  public void actionRegistryListConfigs(ActionRegistryArgs registryArgs)
-      throws YarnException, IOException {
-    ServiceInstanceData instance = lookupInstance(registryArgs);
-
-    RegistryRetriever retriever = new RegistryRetriever(instance);
-    PublishedConfigSet configurations =
-        retriever.getConfigurations(!registryArgs.internal);
-
-    for (String configName : configurations.keys()) {
-      if (!registryArgs.verbose) {
-        log.info("{}", configName);
-      } else {
-        PublishedConfiguration published =
-            configurations.get(configName);
-        log.info("{} : {}",
-            configName,
-            published.description);
-      }
-    }
-  }
  /**
    * list configs available for an instance
    *
@@ -2582,31 +2529,6 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    * @throws FileNotFoundException if the config is not found
    */
   @VisibleForTesting
-  @Deprecated
-  public PublishedConfiguration actionRegistryGetConfig(ActionRegistryArgs registryArgs)
-      throws YarnException, IOException {
-    ServiceInstanceData instance = lookupInstance(registryArgs);
-
-    RegistryRetriever retriever = new RegistryRetriever(instance);
-    boolean external = !registryArgs.internal;
-    PublishedConfigSet configurations =
-        retriever.getConfigurations(external);
-
-    PublishedConfiguration published = retriever.retrieveConfiguration(configurations,
-            registryArgs.getConf,
-            external);
-    return published;
-  }
-
-  /**
-   * list configs available for an instance
-   *
-   * @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 actionRegistryGetConfigYarn(ActionRegistryArgs registryArgs)
       throws YarnException, IOException {
     ServiceRecord instance = lookupServiceRecord(registryArgs);
@@ -2663,96 +2585,49 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   /**
    * Look up an instance
    * @return instance data
-   * @throws UnknownApplicationInstanceException no match
    * @throws SliderException other failures
    * @throws IOException IO problems or wrapped exceptions
    */
-  private ServiceInstanceData lookupInstance(ActionRegistryArgs registryArgs) throws
-      UnknownApplicationInstanceException,
+  private ServiceRecord lookupServiceRecord(ActionRegistryArgs registryArgs) throws
       SliderException,
       IOException {
-    return lookupInstance(registryArgs.name, registryArgs.serviceType);
+    return lookupServiceRecord(registryArgs.serviceType, registryArgs.name);
   }
 
   /**
    * Look up an instance
-   * @param id instance ID
    * @param serviceType service type
+   * @param id instance ID
    * @return instance data
    * @throws UnknownApplicationInstanceException no match
    * @throws SliderException other failures
    * @throws IOException IO problems or wrapped exceptions
    */
-  private ServiceInstanceData lookupInstance(String id,
-      String serviceType) throws
-      IOException {
+  public ServiceRecord lookupServiceRecord(String serviceType, String id)
+      throws IOException, SliderException {
     try {
-      CuratorServiceInstance<ServiceInstanceData> csi =
-          getRegistry().queryForInstance(serviceType, id);
-      if (csi == null) {
-        throw new FileNotFoundException(
-            String.format("instance %s of type %s not found",
-            id, serviceType));
-      }
-      return csi.getPayload();
-    } catch (IOException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new IOException(e);
+      return getRegistryOperations().resolve(
+          servicePath(currentUser(),
+              serviceType, id));
+    } catch (PathNotFoundException e) {
+      throw new UnknownApplicationInstanceException(e.getPath().toString(), e);
     }
-  }
-
-
-  /**
-   * Look up an instance
-   * @return instance data
-   * @throws SliderException other failures
-   * @throws IOException IO problems or wrapped exceptions
-   */
-  private ServiceRecord lookupServiceRecord(ActionRegistryArgs registryArgs) throws
-      SliderException,
-      IOException {
-    return lookupServiceRecord(registryArgs.name, registryArgs.serviceType);
-  }
-
-  /**
-   * Look up an instance
-   * @param id instance ID
-   * @param serviceType service type
-   * @return instance data
-   * @throws UnknownApplicationInstanceException no match
-   * @throws SliderException other failures
-   * @throws IOException IO problems or wrapped exceptions
-   */
-  private ServiceRecord lookupServiceRecord(String id,
-      String serviceType) throws
-      IOException, SliderException {
-    return getRegistryOperations().resolve(
-        BindingUtils.servicePath(BindingUtils.currentUser(),
-            serviceType, id));
   } 
   
-  
-  
   /**
-   * List instances in the registry
-   * @return
-   * @throws IOException
+   * List instances in the registry for the current user
+   * @return a list of slider registry instances
+   * @throws IOException Any IO problem ... including no path in the registry
+   * to slider service classes for this user
    * @throws YarnException
    */
-  public List<CuratorServiceInstance<ServiceInstanceData>> listRegistryInstances()
-      throws IOException, YarnException {
-    return getRegistry().listInstances(SliderKeys.APP_TYPE);
-  }
-
 
-  /**
-   * Get an on-demand path jumper
-   * @return a class that can dump the contents of the registry
-   */
-  @VisibleForTesting
-  public ZKPathDumper dumpSliderRegistry(boolean verbose) throws SliderException, IOException {
-    return getRegistry().dumpPath(verbose);
+  public Map<String, ServiceRecord> listRegistryInstances()
+      throws IOException, YarnException {
+    Map<String, ServiceRecord> recordMap = listServiceRecords(
+        getRegistryOperations(),
+        serviceclassPath(currentUser(), SliderKeys.APP_TYPE));
+    return recordMap;
   }
   
   /**
@@ -2765,7 +2640,10 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       IOException,
       YarnException {
     try {
-      return getRegistry().instanceIDs(SliderKeys.APP_TYPE);
+      Map<String, ServiceRecord> recordMap = listServiceRecords(
+          getRegistryOperations(),
+          serviceclassPath(currentUser(), SliderKeys.APP_TYPE));
+      return new ArrayList<String>(recordMap.keySet());
 /// JDK7    } catch (YarnException | IOException e) {
     } catch (IOException e) {
       throw e;
@@ -2782,39 +2660,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    * @throws SliderException
    * @throws IOException
    */
-  private synchronized SliderRegistryService maybeStartRegistry() throws
-      SliderException,
-      IOException {
-
-    if (registry == null) {
-      registry = startRegistrationService();
-    }
-    return registry;
-  }
-
-  /**
-   * Get the registry binding. As this may start the registry, it can take time
-   * and fail
-   * @return registry the registry service
-   * @throws SliderException slider-specific failures
-   * @throws IOException other failures
-   */
-  @VisibleForTesting
-
-  public SliderRegistryService getRegistry() throws
-      SliderException,
-      IOException {
-    return maybeStartRegistry();
-  }
-
-
-  /**
-   * Start the registry if it is not there yet
-   * @return the registry service
-   * @throws SliderException
-   * @throws IOException
-   */
-  private synchronized RegistryOperationsService maybeStartYarnRegistry()
+  private synchronized RegistryOperations maybeStartYarnRegistry()
       throws SliderException, IOException {
 
     if (registryOperations == null) {
@@ -2824,26 +2670,15 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   }
 
   /**
-   * Get the YARN registry
+   * Get the registry binding. As this may start the registry, it can take time
+   * and fail
    * @return the registry 
    */
-  public RegistryOperationsService getRegistryOperations()
+  public RegistryOperations getRegistryOperations()
       throws SliderException, IOException {
     return maybeStartYarnRegistry();
   }
 
-
-  /**
-   * Get an on-demand path jumper
-   * @return a class that can dump the contents of the registry
-   */
-  @VisibleForTesting
-  public ZKPathDumper dumpYarnRegistry(boolean verbose)
-      throws SliderException, IOException {
-    return getRegistryOperations().dumpPath();
-  }
-
-
   /**
    * Output to standard out/stderr (implementation specific detail)
    * @param src source

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/client/SliderClusterOperations.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClusterOperations.java b/slider-core/src/main/java/org/apache/slider/client/SliderClusterOperations.java
index 9e1f568..30f17b6 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClusterOperations.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClusterOperations.java
@@ -56,7 +56,6 @@ public class SliderClusterOperations {
 
   /**
    * Get a node from the AM
-   * @param appMaster AM
    * @param uuid uuid of node
    * @return deserialized node
    * @throws IOException IO problems

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java b/slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
index 366fc8b..f343933 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
@@ -52,6 +52,7 @@ import java.util.Map;
 public abstract class AbstractLauncher extends Configured {
   private static final Logger log =
     LoggerFactory.getLogger(AbstractLauncher.class);
+  public static final String CLASSPATH = "CLASSPATH";
   /**
    * Filesystem to use for the launch
    */
@@ -194,8 +195,6 @@ public abstract class AbstractLauncher extends Configured {
     dumpLocalResources();
     containerLaunchContext.setLocalResources(localResources);
 
-
-
     //tokens
     log.debug("{} tokens", credentials.numberOfTokens());
     DataOutputBuffer dob = new DataOutputBuffer();
@@ -203,7 +202,6 @@ public abstract class AbstractLauncher extends Configured {
     ByteBuffer tokenBuffer = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
     containerLaunchContext.setTokens(tokenBuffer);
 
-
     return containerLaunchContext;
   }
 
@@ -248,7 +246,6 @@ public abstract class AbstractLauncher extends Configured {
   public void extractResourceRequirements(Resource resource,
                                           Map<String, String> map) {
 
-
     if (map != null) {
       MapOperations options = new MapOperations("", map);
       resource.setMemory(options.getOptionInt(ResourceKeys.YARN_MEMORY,
@@ -258,13 +255,12 @@ public abstract class AbstractLauncher extends Configured {
     }
   }
 
-
   /**
    * Utility method to set up the classpath
    * @param classpath classpath to use
    */
   public void setClasspath(ClasspathConstructor classpath) {
-    setEnv("CLASSPATH", classpath.buildClasspath());
+    setEnv(CLASSPATH, classpath.buildClasspath());
   }
   public void setEnv(String var, String value) {
     Preconditions.checkArgument(var != null, "null variable name");

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java b/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
index 0367e06..0b3fa10 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
@@ -32,7 +32,6 @@ public class JavaCommandLineBuilder extends CommandLineBuilder {
     add(getJavaBinary());
   }
 
-
   /**
    * Get the java binary. This is called in the constructor so don't try and
    * do anything other than return a constant.
@@ -53,6 +52,9 @@ public class JavaCommandLineBuilder extends CommandLineBuilder {
     }
   }
 
+  /**
+   * Turn Java assertions on
+   */
   public void enableJavaAssertions() {
     add("-ea");
     add("-esa");

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java b/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
index d2658c1..65c122f 100644
--- a/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
+++ b/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
@@ -25,6 +25,7 @@ public class CustomRegistryConstants {
 
   public static final String MANAGEMENT_REST_API =
       "org.apache.slider.management";
+  
   public static final String REGISTRY_REST_API =
       "org.apache.slider.registry";
   

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/core/registry/info/ServiceInstanceData.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/registry/info/ServiceInstanceData.java b/slider-core/src/main/java/org/apache/slider/core/registry/info/ServiceInstanceData.java
index c3c7e63..52ae2e2 100644
--- a/slider-core/src/main/java/org/apache/slider/core/registry/info/ServiceInstanceData.java
+++ b/slider-core/src/main/java/org/apache/slider/core/registry/info/ServiceInstanceData.java
@@ -35,6 +35,7 @@ import java.util.Map;
  */
 @JsonIgnoreProperties(ignoreUnknown = true)
 @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@Deprecated
 public class ServiceInstanceData implements Serializable {
 
   public String serviceType;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java b/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
index a05d5be..1d11ef3 100644
--- a/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
+++ b/slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
@@ -34,7 +34,6 @@ import org.apache.slider.core.exceptions.ExceptionConverter;
 import org.apache.slider.core.registry.docstore.PublishedConfigSet;
 import org.apache.slider.core.registry.docstore.PublishedConfiguration;
 import org.apache.slider.core.registry.info.CustomRegistryConstants;
-import org.apache.slider.core.registry.info.ServiceInstanceData;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -63,13 +62,6 @@ public class RegistryRetriever {
     jerseyClient = Client.create(clientConfig);
     jerseyClient.setFollowRedirects(true);
   }
-
-
-  public RegistryRetriever(ServiceInstanceData instance) {
-    externalConfigurationURL =  instance.externalView.configurationsURL;    
-    internalConfigurationURL =  instance.internalView.configurationsURL;    
-  }
-
   
   public RegistryRetriever(String externalConfigurationURL, String internalConfigurationURL) {
     this.externalConfigurationURL = externalConfigurationURL; 
@@ -95,10 +87,8 @@ public class RegistryRetriever {
     } else {
       externalConfigurationURL = "";
     }
-
   }
 
-
   /**
    * Does a bonded registry retriever have a configuration?
    * @param external flag to indicate that it is the external entries to fetch

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java b/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
index 0e7ea25..c223901 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
@@ -23,6 +23,10 @@ import org.apache.hadoop.service.Service;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.registry.client.binding.RegistryTypeUtils;
+import org.apache.hadoop.yarn.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.yarn.registry.client.types.AddressTypes;
+import org.apache.hadoop.yarn.registry.client.types.Endpoint;
 import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
 import org.apache.slider.api.ClusterDescription;
 import org.apache.slider.common.SliderKeys;
@@ -33,14 +37,11 @@ import org.apache.slider.core.conf.AggregateConf;
 import org.apache.slider.core.exceptions.BadCommandArgumentsException;
 import org.apache.slider.core.exceptions.SliderException;
 import org.apache.slider.core.main.ExitCodeProvider;
-import org.apache.slider.core.registry.info.RegisteredEndpoint;
-import org.apache.slider.core.registry.info.ServiceInstanceData;
 import org.apache.slider.server.appmaster.actions.QueueAccess;
 import org.apache.slider.server.appmaster.state.ContainerReleaseSelector;
 import org.apache.slider.server.appmaster.state.MostRecentContainerReleaseSelector;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
 import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
-import org.apache.slider.server.services.registry.RegistryViewForProviders;
 import org.apache.slider.server.services.workflow.ForkedProcessService;
 import org.apache.slider.server.services.workflow.ServiceParent;
 import org.apache.slider.server.services.workflow.WorkflowSequenceService;
@@ -50,6 +51,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.MalformedURLException;
 import java.net.URL;
 import java.util.Collection;
 import java.util.HashMap;
@@ -72,8 +74,6 @@ public abstract class AbstractProviderService
     LoggerFactory.getLogger(AbstractProviderService.class);
   protected StateAccessForProviders amState;
   protected AgentRestOperations restOps;
-  protected RegistryViewForProviders registry;
-  protected ServiceInstanceData registryInstanceData;
   protected URL amWebAPI;
   protected YarnRegistryViewForProviders yarnRegistry;
   protected QueueAccess queueAccess;
@@ -102,11 +102,9 @@ public abstract class AbstractProviderService
   
   @Override
   public void bind(StateAccessForProviders stateAccessor,
-      RegistryViewForProviders reg,
       QueueAccess queueAccess,
       List<Container> liveContainers) {
     this.amState = stateAccessor;
-    this.registry = reg;
     this.queueAccess = queueAccess;
   }
 
@@ -325,27 +323,35 @@ public abstract class AbstractProviderService
 
   @Override
   public void buildEndpointDetails(Map<String, String> details) {
-      ServiceInstanceData self = registry.getSelfRegistration();
-    Map<String, RegisteredEndpoint> endpoints =
-        self.getRegistryView(true).endpoints;
-    for (Map.Entry<String, RegisteredEndpoint> endpoint : endpoints.entrySet()) {
-      RegisteredEndpoint val = endpoint.getValue();
-      if (val.type.equals(RegisteredEndpoint.TYPE_URL)) {
-          details.put(val.description, val.address);
+    ServiceRecord self = yarnRegistry.getSelfRegistration();
+
+    List<Endpoint> externals = self.external;
+    for (Endpoint endpoint : externals) {
+      String addressType = endpoint.addressType;
+      if (AddressTypes.ADDRESS_URI.equals(addressType)) {
+        try {
+          List<URL> urls = RegistryTypeUtils.retrieveAddressURLs(endpoint);
+          if (!urls.isEmpty()) {
+            details.put(endpoint.api, urls.get(0).toString());
+          }
+        } catch (InvalidRecordException ignored) {
+          // Ignored
+        } catch (MalformedURLException ignored) {
+          // ignored
+        }
+
       }
+
     }
   }
 
   @Override
   public void applyInitialRegistryDefinitions(URL amWebURI,
-                                              URL agentOpsURI,
-                                              URL agentStatusURI,
-                                              ServiceInstanceData registryInstanceData,
-                                              ServiceRecord serviceRecord)
+      URL agentOpsURI,
+      URL agentStatusURI,
+      ServiceRecord serviceRecord)
     throws IOException {
-
       this.amWebAPI = amWebURI;
-    this.registryInstanceData = registryInstanceData;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java b/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
index b28ac6a..d78ca0f 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
@@ -32,13 +32,11 @@ import org.apache.slider.core.exceptions.BadCommandArgumentsException;
 import org.apache.slider.core.exceptions.SliderException;
 import org.apache.slider.core.launch.ContainerLauncher;
 import org.apache.slider.core.main.ExitCodeProvider;
-import org.apache.slider.core.registry.info.ServiceInstanceData;
 import org.apache.slider.server.appmaster.actions.QueueAccess;
 import org.apache.slider.server.appmaster.operations.RMOperationHandlerActions;
 import org.apache.slider.server.appmaster.state.ContainerReleaseSelector;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
 import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
-import org.apache.slider.server.services.registry.RegistryViewForProviders;
 import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders;
 
 import java.io.File;
@@ -160,7 +158,6 @@ public interface ProviderService extends ProviderCore,
   Map<String, String> buildMonitorDetails(ClusterDescription clusterSpec);
 
   public void bind(StateAccessForProviders stateAccessor,
-      RegistryViewForProviders reg,
       QueueAccess queueAccess,
       List<Container> liveContainers);
 
@@ -187,14 +184,12 @@ public interface ProviderService extends ProviderCore,
    * @param amWebURI
    * @param agentOpsURI
    * @param agentStatusURI
-   * @param registryInstanceData
    * @param serviceRecord
    */
   void applyInitialRegistryDefinitions(URL amWebURI,
-                                       URL agentOpsURI,
-                                       URL agentStatusURI,
-                                       ServiceInstanceData registryInstanceData,
-                                       ServiceRecord serviceRecord)
+      URL agentOpsURI,
+      URL agentStatusURI,
+      ServiceRecord serviceRecord)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
index 00a1b78..5de08dc 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
@@ -51,8 +51,6 @@ import org.apache.slider.core.launch.CommandLineBuilder;
 import org.apache.slider.core.launch.ContainerLauncher;
 import org.apache.slider.core.registry.docstore.PublishedConfiguration;
 import org.apache.slider.core.registry.info.CustomRegistryConstants;
-import org.apache.slider.core.registry.info.RegisteredEndpoint;
-import org.apache.slider.core.registry.info.ServiceInstanceData;
 import org.apache.slider.providers.AbstractProviderService;
 import org.apache.slider.providers.ProviderCompleted;
 import org.apache.slider.providers.ProviderCore;
@@ -629,31 +627,19 @@ public class AgentProviderService extends AbstractProviderService implements
   }
 
   @Override
-  public void applyInitialRegistryDefinitions (URL amWebURI,
-        URL agentOpsURI,
-        URL agentStatusURI,
-        ServiceInstanceData instanceData,
-        ServiceRecord serviceRecord)
+  public void applyInitialRegistryDefinitions(URL amWebURI,
+      URL agentOpsURI,
+      URL agentStatusURI,
+      ServiceRecord serviceRecord)
     throws IOException {
     super.applyInitialRegistryDefinitions(amWebURI,
                                           agentOpsURI,
                                           agentStatusURI,
-                                          instanceData,
                                           serviceRecord);
 
     try {
       URL restURL = new URL(agentOpsURI, SLIDER_PATH_AGENTS);
-
-      instanceData.internalView.endpoints.put(
-          CustomRegistryConstants.AGENT_SECURE_REST_API,
-          new RegisteredEndpoint(restURL,
-              "Agent Secure REST API"));
       URL agentStatusURL = new URL(agentStatusURI, SLIDER_PATH_AGENTS);
-      instanceData.internalView.endpoints.put(
-          CustomRegistryConstants.AGENT_ONEWAY_REST_API,
-          new RegisteredEndpoint(
-              agentStatusURL,
-              "Agent Oneway REST API"));
 
       serviceRecord.addInternalEndpoint(
           new Endpoint(CustomRegistryConstants.AGENT_SECURE_REST_API,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java b/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
index f01f995..72322c7 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
@@ -36,9 +36,6 @@ import org.apache.slider.core.exceptions.SliderException;
 import org.apache.slider.core.launch.ContainerLauncher;
 import org.apache.slider.core.registry.docstore.PublishedConfiguration;
 import org.apache.slider.core.registry.info.CustomRegistryConstants;
-import org.apache.slider.core.registry.info.RegisteredEndpoint;
-import org.apache.slider.core.registry.info.RegistryView;
-import org.apache.slider.core.registry.info.ServiceInstanceData;
 import org.apache.slider.providers.AbstractProviderService;
 import org.apache.slider.providers.ProviderCompleted;
 import org.apache.slider.providers.ProviderCore;
@@ -56,8 +53,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_MANAGEMENT;
-import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_PUBLISHER;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
 
 /**
  * Exists just to move some functionality out of AppMaster into a peer class
@@ -115,13 +111,11 @@ public class SliderAMProviderService extends AbstractProviderService implements
   public void applyInitialRegistryDefinitions(URL amWebURI,
       URL agentOpsURI,
       URL agentStatusURI,
-      ServiceInstanceData instanceData,
       ServiceRecord serviceRecord)
       throws IOException {
     super.applyInitialRegistryDefinitions(amWebURI,
         agentOpsURI,
         agentStatusURI,
-        instanceData,
         serviceRecord);
     // now publish site.xml files
     YarnConfiguration defaultYarnConfig = new YarnConfiguration();
@@ -129,8 +123,7 @@ public class SliderAMProviderService extends AbstractProviderService implements
         PublishedArtifacts.COMPLETE_CONFIG,
         new PublishedConfiguration(
             "Complete slider application settings",
-            getConfig(), getConfig())
-    );
+            getConfig(), getConfig()));
     amState.getPublishedSliderConfigurations().put(
         PublishedArtifacts.YARN_SITE_CONFIG,
         new PublishedConfiguration(
@@ -153,40 +146,16 @@ public class SliderAMProviderService extends AbstractProviderService implements
 
 
     try {
-      RegisteredEndpoint webUI =
-          new RegisteredEndpoint(amWebURI, "Application Master Web UI");
 
       URL managementAPI = new URL(amWebURI, SLIDER_PATH_MANAGEMENT);
-      URL registryREST = new URL(amWebURI, RestPaths.SLIDER_PATH_REGISTRY + "/" +
-                                                 RestPaths.REGISTRY_SERVICE);
-
-      RegistryView externalView = instanceData.externalView;
-      externalView.endpoints.put(CustomRegistryConstants.WEB_UI, webUI);
-
-      externalView.endpoints.put(
-          CustomRegistryConstants.MANAGEMENT_REST_API,
-          new RegisteredEndpoint(
-              managementAPI,
-              "Management REST API") );
-
-      externalView.endpoints.put(
-          CustomRegistryConstants.REGISTRY_REST_API,
-          new RegisteredEndpoint(
-              registryREST,
-              "Registry Web Service" ) );
+      URL registryREST = new URL(amWebURI, SLIDER_PATH_REGISTRY );
 
       URL publisherURL = new URL(amWebURI, SLIDER_PATH_PUBLISHER);
-      externalView.endpoints.put(
-          CustomRegistryConstants.PUBLISHER_REST_API,
-          new RegisteredEndpoint(
-              publisherURL,
-              "Publisher Service") );
 
       // Set the configurations URL.
 
       String configurationsURL = SliderUtils.appendToURL(
           publisherURL.toExternalForm(), RestPaths.SLIDER_CONFIGSET);
-      externalView.configurationsURL = configurationsURL;
 
       serviceRecord.addExternalEndpoint(
           RegistryTypeUtils.webEndpoint(

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 754013a..df54cfa 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -52,8 +52,8 @@ import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.registry.client.api.RegistryOperations;
 import org.apache.hadoop.yarn.registry.client.binding.RegistryPathUtils;
-import org.apache.hadoop.yarn.registry.client.services.RegistryOperationsService;
 import org.apache.hadoop.yarn.registry.client.types.PersistencePolicies;
 import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
 import org.apache.hadoop.yarn.registry.client.binding.RegistryTypeUtils;
@@ -95,9 +95,7 @@ import org.apache.slider.core.main.RunService;
 import org.apache.slider.core.main.ServiceLauncher;
 import org.apache.slider.core.persist.ConfTreeSerDeser;
 import org.apache.slider.core.registry.info.CustomRegistryConstants;
-import org.apache.slider.core.registry.info.RegisteredEndpoint;
 import org.apache.slider.core.registry.info.RegistryNaming;
-import org.apache.slider.core.registry.info.ServiceInstanceData;
 import org.apache.slider.providers.ProviderCompleted;
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.providers.ProviderService;
@@ -137,7 +135,6 @@ import org.apache.slider.server.appmaster.web.SliderAMWebApp;
 import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.apache.slider.server.appmaster.web.WebAppApiImpl;
 import org.apache.slider.server.appmaster.web.rest.RestPaths;
-import org.apache.slider.server.services.registry.SliderRegistryService;
 import org.apache.slider.server.services.security.CertificateManager;
 import org.apache.slider.server.services.security.FsDelegationTokenManager;
 import org.apache.slider.server.services.utility.AbstractSliderLaunchedService;
@@ -182,6 +179,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     ServiceStateChangeListener,
     RoleKeys,
     ProviderCompleted {
+  
   protected static final Logger log =
     LoggerFactory.getLogger(SliderAppMaster.class);
 
@@ -204,8 +202,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
   public static final int HEARTBEAT_INTERVAL = 1000;
   public static final int NUM_RPC_HANDLERS = 5;
-  public static final String SLIDER_AM_RPC = "Slider AM RPC";
-  public static final int SCHEDULED_EXECUTOR_POOL_SIZE = 1;
 
   /**
    * Singleton of metrics registry
@@ -327,16 +323,10 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   private ProviderService providerService;
 
   /**
-   * The registry service
-   */
-  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-  private SliderRegistryService registry;
-  
-  /**
    * The YARN registry service
    */
   @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-  private RegistryOperationsService registryOperations;
+  private RegistryOperations registryOperations;
 
   /**
    * Record of the max no. of cores allowed in this cluster
@@ -648,12 +638,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       appInformation.put(StatusKeys.INFO_AM_HOSTNAME, appMasterHostname);
       appInformation.set(StatusKeys.INFO_AM_RPC_PORT, appMasterRpcPort);
 
-      
-      //registry
-      log.info("Starting slider registry");
-      registry = startRegistrationService();
-      log.info(registry.toString());
-
       log.info("Starting Yarn registry");
       registryOperations = startRegistryOperationsService();
       log.info(registryOperations.toString());
@@ -671,12 +655,12 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
       startAgentWebApp(appInformation, serviceConf);
 
-      webApp = new SliderAMWebApp(registry, registryOperations);
+      webApp = new SliderAMWebApp(registryOperations);
       WebApps.$for(SliderAMWebApp.BASE_PATH, WebAppApi.class,
                    new WebAppApiImpl(this,
                                      stateForProviders,
                                      providerService,
-                                     certificateManager),
+                                     certificateManager, registryOperations),
                    RestPaths.WS_CONTEXT)
                       .with(serviceConf)
                       .start(webApp);
@@ -790,9 +774,9 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
 
     //Give the provider restricted access to the state, registry
-    providerService.bind(stateForProviders, registry, actionQueues,
+    providerService.bind(stateForProviders, actionQueues,
         liveContainers);
-    sliderAMProvider.bind(stateForProviders, registry, actionQueues,
+    sliderAMProvider.bind(stateForProviders, actionQueues,
         liveContainers);
 
     // chaos monkey
@@ -873,7 +857,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
                      new WebAppApiImpl(this,
                                        stateForProviders,
                                        providerService,
-                                       certificateManager),
+                                       certificateManager, registryOperations),
                      RestPaths.AGENT_WS_CONTEXT)
         .withComponentConfig(getInstanceDefinition().getAppConfOperations()
                                  .getComponent(SliderKeys.COMPONENT_AM))
@@ -930,22 +914,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     providerService.bindToYarnRegistry(yarnRegistryOperations);
     sliderAMProvider.bindToYarnRegistry(yarnRegistryOperations);
 
-    List<String> serviceInstancesRunning = registry.instanceIDs(serviceName);
-    log.info("service instances already running: {}", serviceInstancesRunning);
-
-
-    // slider instance data
-    ServiceInstanceData instanceData = new ServiceInstanceData(registryId,
-        serviceType);
-
-
-    // IPC services
-    instanceData.externalView.endpoints.put(
-        CustomRegistryConstants.AM_IPC_PROTOCOL,
-        new RegisteredEndpoint(rpcServiceAddress,
-            RegisteredEndpoint.PROTOCOL_HADOOP_PROTOBUF,
-            SLIDER_AM_RPC) );
-
     // Yarn registry
     ServiceRecord serviceRecord = new ServiceRecord();
     String serviceID = appid.toString();
@@ -960,38 +928,34 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
             RegistryTypeUtils.marshall(rpcServiceAddress)));
     
     // internal services
-
-
     sliderAMProvider.applyInitialRegistryDefinitions(amWebURI,
-                                                     agentOpsURI,
-                                                     agentStatusURI,
-                                                     instanceData,
-                                                     serviceRecord);
+        agentOpsURI,
+        agentStatusURI,
+        serviceRecord);
 
     // provider service dynamic definitions.
     providerService.applyInitialRegistryDefinitions(amWebURI,
-                                                    agentOpsURI,
-                                                    agentStatusURI,
-                                                    instanceData,
-                                                    serviceRecord);
-
+        agentOpsURI,
+        agentStatusURI,
+        serviceRecord);
 
-    // push the registration info to ZK
-    registry.registerSelf(
-        instanceData, amWebURI
-    );
 
+    // store for clients
     log.info("Service Record \n{}", serviceRecord);
     yarnRegistryOperations.putService(service_user_name,
         SliderKeys.APP_TYPE,
         instanceName,
         serviceRecord);
+    yarnRegistryOperations.setSelfRegistration(serviceRecord);
 
-    // and an ephemeral binding to the app
+    // and a shorter lived binding to the app
+    String attempt = appAttemptID.toString();
+    ServiceRecord attemptRecord = new ServiceRecord(serviceRecord);
+    attemptRecord.id = attempt;
+    attemptRecord.persistence = PersistencePolicies.APPLICATION_ATTEMPT;
     yarnRegistryOperations.putComponent(
-        RegistryPathUtils.encodeYarnID(appAttemptID.toString()),
-        serviceRecord
-    );
+        RegistryPathUtils.encodeYarnID(attempt),
+        serviceRecord);
 
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
index 36e1dad..64be603 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
@@ -22,6 +22,7 @@ import com.sun.jersey.api.core.ResourceConfig;
 import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
 import com.sun.jersey.spi.container.servlet.ServletContainer;
 import org.apache.curator.x.discovery.ServiceDiscovery;
+import org.apache.hadoop.yarn.registry.client.api.RegistryOperations;
 import org.apache.hadoop.yarn.registry.client.services.RegistryOperationsService;
 import org.apache.hadoop.yarn.webapp.Dispatcher;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
@@ -30,6 +31,7 @@ import org.apache.slider.core.registry.info.ServiceInstanceData;
 import org.apache.slider.server.appmaster.web.rest.AMWadlGeneratorConfig;
 import org.apache.slider.server.appmaster.web.rest.AMWebServices;
 import org.apache.slider.server.appmaster.web.rest.SliderJacksonJaxbJsonProvider;
+import org.apache.slider.server.appmaster.web.rest.registry.RegistryResource;
 import org.apache.slider.server.services.curator.CuratorHelper;
 import org.apache.slider.server.services.registry.RegistryDiscoveryContext;
 import org.apache.slider.server.services.registry.RegistryRestResources;
@@ -48,13 +50,9 @@ public class SliderAMWebApp extends WebApp {
   public static final String CONTAINER_STATS = "/stats";
   public static final String CLUSTER_SPEC = "/spec";
 
-  public final SliderRegistryService registry;
-  private final RegistryOperationsService yarnRegistry;
+  private final RegistryOperations registry;
 
-
-  public SliderAMWebApp(SliderRegistryService registry,
-      RegistryOperationsService yarnRegistry) {
-    this.yarnRegistry = yarnRegistry;
+  public SliderAMWebApp(RegistryOperations registry) {
     Preconditions.checkArgument(registry != null, "registry null");
     this.registry = registry;
   }
@@ -71,18 +69,6 @@ public class SliderAMWebApp extends WebApp {
     // bind the REST interface
     bind(AMWebServices.class);
     //bind(AMAgentWebServices.class);
-
-    CuratorHelper curatorHelper = registry.getCuratorHelper();
-    ServiceDiscovery<ServiceInstanceData> discovery = registry.getDiscovery();
-    RegistryDiscoveryContext discoveryContext = curatorHelper
-                                                        .createDiscoveryContext(
-                                                          discovery);
-
-    bind(RegistryDiscoveryContext.class).toInstance(discoveryContext);
-    RegistryRestResources registryRestResources =
-      new RegistryRestResources(discoveryContext, registry);
-    bind(RegistryRestResources.class).toInstance(registryRestResources);
-
     route("/", SliderAMController.class);
     route(CONTAINER_STATS, SliderAMController.class, "containerStats");
     route(CLUSTER_SPEC, SliderAMController.class, "specification");

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
index aa20baa..8d91fd4 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
@@ -16,6 +16,7 @@
  */
 package org.apache.slider.server.appmaster.web;
 
+import org.apache.hadoop.yarn.registry.client.api.RegistryOperations;
 import org.apache.slider.api.SliderClusterProtocol;
 import org.apache.slider.providers.ProviderService;
 import org.apache.slider.server.appmaster.state.AppState;
@@ -62,4 +63,7 @@ public interface WebAppApi {
    * Returns an interface that can support the agent-based REST operations.
    */
   public AgentRestOperations getAgentRestOperations();
+
+
+  RegistryOperations getRegistryOperations();
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
index 4d595a9..52b53a9 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
@@ -16,6 +16,7 @@
  */
 package org.apache.slider.server.appmaster.web;
 
+import org.apache.hadoop.yarn.registry.client.api.RegistryOperations;
 import org.apache.slider.api.SliderClusterProtocol;
 import org.apache.slider.common.SliderKeys;
 import org.apache.slider.providers.ProviderRole;
@@ -47,11 +48,14 @@ public class WebAppApiImpl implements WebAppApi {
   protected final StateAccessForProviders appState;
   protected final ProviderService provider;
   protected final CertificateManager certificateManager;
-  
+  private final RegistryOperations registryOperations;
+
   public WebAppApiImpl(SliderClusterProtocol clusterProto,
-                       StateAccessForProviders appState,
-                       ProviderService provider,
-                       CertificateManager certificateManager) {
+      StateAccessForProviders appState,
+      ProviderService provider,
+      CertificateManager certificateManager,
+      RegistryOperations registryOperations) {
+    this.registryOperations = registryOperations;
     checkNotNull(clusterProto);
     checkNotNull(appState);
     checkNotNull(provider);
@@ -145,4 +149,9 @@ public class WebAppApiImpl implements WebAppApi {
   public AgentRestOperations getAgentRestOperations() {
     return provider.getAgentRestOperations();
   }
+
+  @Override
+  public RegistryOperations getRegistryOperations() {
+    return registryOperations;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
index 4f068f3..30db98e 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
@@ -22,6 +22,7 @@ import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.apache.slider.server.appmaster.web.rest.agent.AgentResource;
 import org.apache.slider.server.appmaster.web.rest.management.ManagementResource;
 import org.apache.slider.server.appmaster.web.rest.publisher.PublisherResource;
+import org.apache.slider.server.appmaster.web.rest.registry.RegistryResource;
 
 import javax.ws.rs.Path;
 
@@ -29,6 +30,7 @@ import javax.ws.rs.Path;
 @Singleton
 @Path(RestPaths.SLIDER_CONTEXT_ROOT)
 public class AMWebServices {
+  
   /** AM/WebApp info object */
   private WebAppApi slider;
 
@@ -46,4 +48,11 @@ public class AMWebServices {
   public PublisherResource getPublisherResource() {
     return new PublisherResource(slider);
   }
+ 
+  @Path(RestPaths.SLIDER_SUBPATH_REGISTRY)
+  public RegistryResource getRegistryResource() {
+    return new RegistryResource(slider);
+  }
+  
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
index 0571ca1..93601ad 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
@@ -42,10 +42,12 @@ public class RestPaths {
                                       + SLIDER_SUBPATH_PUBLISHER;
 
   public static final String SLIDER_SUBPATH_REGISTRY = "/registry";
-  public static final String SLIDER_PATH_REGISTRY = WS_CONTEXT_ROOT
+  public static final String SLIDER_PATH_REGISTRY = SLIDER_CONTEXT_ROOT
                                                     + SLIDER_SUBPATH_REGISTRY;
 
+  @Deprecated
   public static final String REGISTRY_SERVICE = "v1/service";
+  @Deprecated
   public static final String REGISTRY_ANYSERVICE = "v1/anyservice";
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java
new file mode 100644
index 0000000..67efa0d
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java
@@ -0,0 +1,130 @@
+/*
+ * 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.server.appmaster.web.rest.registry;
+
+import com.google.inject.Singleton;
+import org.apache.hadoop.fs.PathAccessDeniedException;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.yarn.registry.client.api.RegistryOperations;
+import org.apache.hadoop.yarn.registry.client.exceptions.AuthenticationFailedException;
+import org.apache.hadoop.yarn.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import java.util.List;
+
+/**
+ * This is the read-only view of the slider YARN registry
+ * 
+ * Model:
+ * <ol>
+ *   <li>a tree of nodes</li>
+ *   <li>Default view is of children + record</li>
+ * </ol>
+ * 
+ */
+@Singleton
+//@Path(RestPaths.SLIDER_PATH_REGISTRY)
+public class RegistryResource {
+  protected static final Logger log =
+      LoggerFactory.getLogger(RegistryResource.class);
+  public static final String SERVICE_PATH =
+      "/{path:.*}";
+
+  private final RegistryOperations registry;
+
+  /**
+   * Construct an instance bonded to a registry
+   * @param slider slider API
+   */
+  public RegistryResource(WebAppApi slider) {
+    this.registry = slider.getRegistryOperations();
+  }
+
+  /**
+   * Internal init code, per request
+   * @param request incoming request 
+   * @param uriInfo URI details
+   */
+  private void init(HttpServletRequest request, UriInfo uriInfo) {
+    log.debug(uriInfo.getRequestUri().toString());
+  }
+
+  @GET
+  public Response getRoot(@Context HttpServletRequest request) {
+    return Response.ok("registry root").build();
+  }
+
+
+//   {path:.*}
+
+  @Path(SERVICE_PATH)
+  @GET
+  @Produces({MediaType.APPLICATION_JSON})
+  public Response lookup(
+      @PathParam("path") String path,
+      @Context HttpServletRequest request,
+      @Context UriInfo uriInfo) {
+    try {
+      init(request, uriInfo);
+      List<RegistryPathStatus> list = registry.listFull(path);
+      return Response.ok("found").build();
+    } catch (PathNotFoundException e) {
+      throw new NotFoundException(path);
+    } catch (AuthenticationFailedException e) {
+      throw new ForbiddenException(path);
+    } catch (PathAccessDeniedException e) {
+      throw new ForbiddenException(path);
+    } catch (Exception e) {
+      return fromException(e);
+    }
+  }
+
+  /**
+   * Handle an exception
+   * @param e exception
+   * @return a response to return
+   */
+  Response fromException(Exception e) {
+    log.error("Error during generation of response: {}", e, e);
+    if (e instanceof PathNotFoundException) {
+      return Response.status(Response.Status.NOT_FOUND).build();
+    }
+    if (e instanceof AuthenticationFailedException
+        || e instanceof PathAccessDeniedException) {
+      return Response.status(Response.Status.FORBIDDEN).build();
+    }
+    return Response.serverError().build();
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorHelper.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorHelper.java b/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorHelper.java
index 72d413f..9e6f53a 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorHelper.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorHelper.java
@@ -75,17 +75,6 @@ public class CuratorHelper extends Configured {
   }
 
   /**
-   * Create an (united) curator client service
-   * @param connectionString ZK binding
-   * @return the service
-   */
-  public CuratorService createCuratorClientService() {
-    CuratorService curatorService =
-      new CuratorService("Curator ", curator, connectionString);
-    return curatorService;
-  }
-
-  /**
    * Create a discovery builder bonded to this curator
    * @return
    */

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java b/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java
index c3ace4d..a98a192 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java
@@ -30,6 +30,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.Closeable;
 
+@Deprecated
 public class CuratorService extends AbstractService {
   private static final Logger log =
     LoggerFactory.getLogger(CuratorService.class);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorServiceInstance.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorServiceInstance.java b/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorServiceInstance.java
index 61efde2..0acadb2 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorServiceInstance.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorServiceInstance.java
@@ -24,6 +24,7 @@ import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.annotate.JsonTypeInfo;
 
 @JsonIgnoreProperties(ignoreUnknown = true)
+@Deprecated
 public class CuratorServiceInstance<T> {
 
   public String name;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorServiceInstances.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorServiceInstances.java b/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorServiceInstances.java
index 8923e63..2b575be 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorServiceInstances.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorServiceInstances.java
@@ -24,6 +24,7 @@ import java.util.List;
 /**
  *
  */
+@Deprecated
 public class CuratorServiceInstances<T> {
   private final List<CuratorServiceInstance<T>> services;
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorUriSpec.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorUriSpec.java b/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorUriSpec.java
index adda359..478d3f4 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorUriSpec.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorUriSpec.java
@@ -26,6 +26,7 @@ import java.util.List;
  *
  */
 @JsonIgnoreProperties(ignoreUnknown = true)
+@Deprecated
 public class CuratorUriSpec extends UriSpec{
 
   private final List<Part> parts = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java b/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java
index dbdcee4..fca967b 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java
@@ -46,6 +46,7 @@ import java.util.Map;
  * start/close methods are tied to the lifecycle of this service
  * @param <Payload> the payload of the operation
  */
+@Deprecated
 public class RegistryBinderService<Payload> extends CuratorService {
   private static final Logger log =
     LoggerFactory.getLogger(RegistryBinderService.class);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryDiscoveryContext.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryDiscoveryContext.java b/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryDiscoveryContext.java
index a7c35e8..d6b676f 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryDiscoveryContext.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryDiscoveryContext.java
@@ -23,6 +23,7 @@ import org.apache.curator.x.discovery.ServiceDiscovery;
 import org.apache.curator.x.discovery.server.contexts.GenericDiscoveryContext;
 import org.apache.slider.core.registry.info.ServiceInstanceData;
 
+@Deprecated
 public class RegistryDiscoveryContext extends GenericDiscoveryContext<ServiceInstanceData> {
 
   public RegistryDiscoveryContext(ServiceDiscovery<ServiceInstanceData> serviceDiscovery,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryRestResources.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryRestResources.java b/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryRestResources.java
index e4e8523..ea97847 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryRestResources.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryRestResources.java
@@ -48,6 +48,7 @@ import java.util.Random;
 
 @Singleton
 @Path(RestPaths.SLIDER_PATH_REGISTRY)
+@Deprecated
 public class RegistryRestResources extends DiscoveryResource<ServiceInstanceData> {
   public static final String SERVICE_NAME = RestPaths.REGISTRY_SERVICE +"/{name}";
   public static final String SERVICE_NAME_ID = SERVICE_NAME + "/{id}";

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryServiceConstants.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryServiceConstants.java b/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryServiceConstants.java
index ee24dc1..f3a107f 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryServiceConstants.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryServiceConstants.java
@@ -21,6 +21,7 @@ package org.apache.slider.server.services.registry;
 /**
  * These constants are unique to the slider registry service itself
  */
+@Deprecated
 public class RegistryServiceConstants {
   public static final int INSTANCE_REFRESH_MS = 1000;
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java b/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java
deleted file mode 100644
index 22ba066..0000000
--- a/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.server.services.registry;
-
-import org.apache.slider.core.registry.info.ServiceInstanceData;
-
-import java.io.IOException;
-import java.net.URL;
-import java.util.List;
-
-/**
- * This offers restricted access to the registry for providers
- */
-public interface RegistryViewForProviders {
-  List<ServiceInstanceData> listInstancesByType(String serviceType) throws
-      IOException;
-
-  /**
-   * Get the registration of slider itself
-   * @return the registration of slider
-   */
-  ServiceInstanceData getSelfRegistration();
-
-  /**
-   * Register the service, raising IOExceptions when anything fails
-   * @param instanceData instance data
-   * @param url URL to register
-   * @throws IOException on registration problems
-   */
-  void registerServiceInstance(
-      ServiceInstanceData instanceData, URL url) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java b/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java
index ca4d180..ab92c82 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java
@@ -34,10 +34,9 @@ import java.util.List;
  * This is the registry service, which tries to hide exactly how the
  * registry is implemented
  */
-
+@Deprecated
 public class SliderRegistryService
-    extends RegistryBinderService<ServiceInstanceData>
-    implements RegistryViewForProviders {
+    extends RegistryBinderService<ServiceInstanceData> {
 
   private ServiceInstanceData selfRegistration;
 
@@ -48,7 +47,6 @@ public class SliderRegistryService
   }
 
 
-  @Override
   public List<ServiceInstanceData> listInstancesByType(String serviceType) throws
       IOException {
     List<CuratorServiceInstance<ServiceInstanceData>> services =
@@ -60,7 +58,6 @@ public class SliderRegistryService
     return payloads;
   }
 
-  @Override
   public ServiceInstanceData getSelfRegistration() {
     return selfRegistration;
   }
@@ -81,7 +78,6 @@ public class SliderRegistryService
     setSelfRegistration(instanceData);
   }
 
-  @Override
   public void registerServiceInstance(
       ServiceInstanceData instanceData, URL url) throws IOException {
     Preconditions.checkNotNull(instanceData);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/services/utility/AbstractSliderLaunchedService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/utility/AbstractSliderLaunchedService.java b/slider-core/src/main/java/org/apache/slider/server/services/utility/AbstractSliderLaunchedService.java
index 74718c9..ada0047 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/utility/AbstractSliderLaunchedService.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/utility/AbstractSliderLaunchedService.java
@@ -20,14 +20,13 @@ package org.apache.slider.server.services.utility;
 
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
-import org.apache.hadoop.yarn.registry.client.services.RegistryOperationsService;
+import org.apache.hadoop.yarn.registry.client.api.RegistryOperations;
+import org.apache.hadoop.yarn.registry.client.api.RegistryOperationsFactory;
 import org.apache.slider.common.SliderXmlConfKeys;
 import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.exceptions.BadCommandArgumentsException;
 import org.apache.slider.core.exceptions.BadConfigException;
 import org.apache.slider.core.zk.ZookeeperUtils;
-import org.apache.slider.server.services.curator.CuratorHelper;
-import org.apache.slider.server.services.registry.SliderRegistryService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -46,21 +45,6 @@ public abstract class AbstractSliderLaunchedService extends
   }
 
   /**
-   * Start the registration service
-   * @return the instance
-   * @throws BadConfigException
-   */
-  protected SliderRegistryService startRegistrationService()
-      throws BadConfigException {
-
-    String registryQuorum = lookupZKQuorum();
-    String zkPath = getConfig().get(
-        SliderXmlConfKeys.REGISTRY_PATH,
-        SliderXmlConfKeys.DEFAULT_REGISTRY_PATH);
-    return startSliderRegistrationService(registryQuorum, zkPath);
-  }
-
-  /**
    * look up the registry quorum from the config
    * @return the quorum string
    * @throws BadConfigException if it is not there or invalid
@@ -86,36 +70,17 @@ public abstract class AbstractSliderLaunchedService extends
   }
 
   /**
-   * Start the registration service
-   * @param zkConnection
-   * @param zkPath
-   * @return
-   */
-  public SliderRegistryService startSliderRegistrationService(
-      String zkConnection,
-      String zkPath) {
-    CuratorHelper curatorHelper =
-      new CuratorHelper(getConfig(), zkConnection);
-
-    //registry will start curator as well as the binder, in the correct order
-    SliderRegistryService registryBinderService =
-      curatorHelper.createRegistryBinderService(zkPath);
-    deployChildService(registryBinderService);
-    return registryBinderService;
-  }
-
-  /**
    * Create, adopt ,and start the YARN registration service
    * @return the registry operations service, already deployed as a child
    * of the AbstractSliderLaunchedService instance.
    */
-  public RegistryOperationsService startRegistryOperationsService()
+  public RegistryOperations startRegistryOperationsService()
       throws BadConfigException {
 
     // push back the slider registry entry if needed
     String quorum = lookupZKQuorum();
     getConfig().set(RegistryConstants.KEY_REGISTRY_ZK_QUORUM, quorum);
-    RegistryOperationsService registryWriterService =
+    RegistryOperations registryWriterService =
         createRegistryOperationsInstance();
     deployChildService(registryWriterService);
     return registryWriterService;
@@ -126,8 +91,8 @@ public abstract class AbstractSliderLaunchedService extends
    * subclasses to instantiate a subclass service
    * @return an instance to match to the lifecycle of this service
    */
-  protected RegistryOperationsService createRegistryOperationsInstance() {
-    return new RegistryOperationsService("YarnRegistry");
+  protected RegistryOperations createRegistryOperationsInstance() {
+    return RegistryOperationsFactory.createInstance("YarnRegistry", getConfig());
   }
 
   /**
@@ -139,9 +104,8 @@ public abstract class AbstractSliderLaunchedService extends
   protected static void requireArgumentSet(String argname, String value)
       throws BadCommandArgumentsException {
     if (isUnset(value)) {
-      throw new BadCommandArgumentsException("Required argument "
-                                             + argname
-                                             + " missing");
+      throw new BadCommandArgumentsException(
+          "Required argument " + argname + " missing");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java b/slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java
index 65ae420..a08cfaf 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java
@@ -18,9 +18,10 @@
 
 package org.apache.slider.server.services.yarnregistry;
 
-import org.apache.hadoop.yarn.registry.client.binding.BindingUtils;
+import org.apache.hadoop.yarn.registry.client.api.RegistryOperations;
+import org.apache.hadoop.yarn.registry.client.binding.RegistryOperationUtils;
 import org.apache.hadoop.yarn.registry.client.binding.RegistryPathUtils;
-import org.apache.hadoop.yarn.registry.client.services.RegistryOperationsService;
+
 import org.apache.hadoop.yarn.registry.client.api.CreateFlags;
 import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
 
@@ -28,14 +29,15 @@ import java.io.IOException;
 
 public class YarnRegistryViewForProviders {
 
-  private final RegistryOperationsService registryOperations;
+  private final RegistryOperations registryOperations;
 
   private final String user;
 
   private final String sliderServiceclass;
   private final String instanceName;
+  private ServiceRecord selfRegistration;
 
-  public YarnRegistryViewForProviders(RegistryOperationsService registryOperations,
+  public YarnRegistryViewForProviders(RegistryOperations registryOperations,
       String user, String sliderServiceclass, String instanceName) {
     this.registryOperations = registryOperations;
     this.user = user;
@@ -55,10 +57,18 @@ public class YarnRegistryViewForProviders {
     return instanceName;
   }
 
-  public RegistryOperationsService getRegistryOperationsService() {
+  public RegistryOperations getRegistryOperations() {
     return registryOperations;
   }
 
+  public ServiceRecord getSelfRegistration() {
+    return selfRegistration;
+  }
+
+  public void setSelfRegistration(ServiceRecord selfRegistration) {
+    this.selfRegistration = selfRegistration;
+  }
+
   /**
    * Add a component under the slider name/entry
    * @param componentName component name
@@ -70,8 +80,7 @@ public class YarnRegistryViewForProviders {
       IOException {
     putComponent(sliderServiceclass, instanceName,
         componentName,
-        record
-    );
+        record);
   }
 
   /**
@@ -85,11 +94,10 @@ public class YarnRegistryViewForProviders {
       String serviceName,
       String componentName,
       ServiceRecord record) throws IOException {
-    String path = BindingUtils.componentPath(
+    String path = RegistryOperationUtils.componentPath(
         user, serviceClass, serviceName, componentName);
     registryOperations.mknode(RegistryPathUtils.parentOf(path), true);
-    registryOperations.create(path, record,
-        CreateFlags.OVERWRITE);
+    registryOperations.create(path, record, CreateFlags.OVERWRITE);
   }
 
 
@@ -106,7 +114,7 @@ public class YarnRegistryViewForProviders {
       ServiceRecord record) throws IOException {
 
     
-    String path = BindingUtils.servicePath(
+    String path = RegistryOperationUtils.servicePath(
         username, serviceClass, serviceName);
     registryOperations.mknode(RegistryPathUtils.parentOf(path), true);
     registryOperations.create(path, record, CreateFlags.OVERWRITE);
@@ -115,7 +123,7 @@ public class YarnRegistryViewForProviders {
 
 
   public void rmComponent(String componentName) throws IOException {
-    String path = BindingUtils.componentPath(
+    String path = RegistryOperationUtils.componentPath(
         user, sliderServiceclass, instanceName,
         componentName);
     registryOperations.delete(path, false);