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:12 UTC

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

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
index 68cac7b..e04f9cb 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
@@ -96,11 +96,6 @@ class TestStandaloneAgentAM  extends AgentMiniClusterTestBase {
     assert instance != null
 
     //switch to the slider ZK-based registry
-
-    describe "service registry names"
-    SliderRegistryService registry = client.registry
-    def names = registry.getServiceTypes();
-    dumpRegistryServiceTypes(names)
     describe "service registry instance IDs"
 
     def instanceIds = client.listRegisteredSliderInstances()
@@ -108,14 +103,6 @@ class TestStandaloneAgentAM  extends AgentMiniClusterTestBase {
     log.info("number of instanceIds: ${instanceIds.size()}")
     instanceIds.each { String it -> log.info(it) }
 
-    describe "service registry slider instances"
-    List<CuratorServiceInstance<ServiceInstanceData>> instances =
-        client.listRegistryInstances()
-    instances.each { CuratorServiceInstance<ServiceInstanceData> svc ->
-      log.info svc.toString()
-    }
-    describe "end list service registry slider instances"
-
     describe "Yarn registry"
     def yarnRegistry = client.registryOperations
     

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneYarnRegistryAM.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneYarnRegistryAM.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneYarnRegistryAM.groovy
index e88fbe8..65bd077 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneYarnRegistryAM.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneYarnRegistryAM.groovy
@@ -20,17 +20,18 @@ package org.apache.slider.agent.standalone
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
-import org.apache.hadoop.fs.PathNotFoundException
 import org.apache.hadoop.yarn.api.records.ApplicationReport
 import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.registry.client.api.RegistryConstants
 import org.apache.hadoop.yarn.registry.client.binding.RecordOperations
 import org.apache.hadoop.yarn.registry.client.binding.RegistryTypeUtils
+import org.apache.hadoop.yarn.registry.client.services.RegistryOperationsClient
 import org.apache.hadoop.yarn.registry.client.types.RegistryPathStatus
 import org.apache.hadoop.yarn.registry.client.types.ServiceRecord
+import org.apache.slider.core.exceptions.UnknownApplicationInstanceException
 
-import static org.apache.hadoop.yarn.registry.client.binding.BindingUtils.*
+import static org.apache.hadoop.yarn.registry.client.binding.RegistryOperationUtils.*
 import org.apache.slider.agent.AgentMiniClusterTestBase
 import org.apache.slider.api.ClusterNode
 import org.apache.slider.client.SliderClient
@@ -73,7 +74,7 @@ class TestStandaloneYarnRegistryAM extends AgentMiniClusterTestBase {
     registryOperations.stat(RegistryConstants.PATH_SYSTEM_SERVICES)
     
     // verify the cluster has the YARN reg service live
-    def rmRegistryService = miniCluster.getResourceManager(0).getRMContext().registry
+    def rmRegistryService = miniCluster.getResourceManager(0).RMContext.registry
     assert rmRegistryService
     
     ServiceLauncher<SliderClient> launcher
@@ -124,48 +125,47 @@ class TestStandaloneYarnRegistryAM extends AgentMiniClusterTestBase {
 
     // sleep to allow registration to complete
     sleep(5000)
-    
 
-    
 
+    describe "service registry names"
+    def registryService = client.registryOperations
+
+    RegistryOperationsClient registryOperationsClient =
+        registryService as RegistryOperationsClient
     try {
-      def yarnRegistryDump = client.dumpYarnRegistry(true).toString()
+      def yarnRegistryDump = registryOperationsClient.dumpPath() 
       log.info("yarn service registry: \n${yarnRegistryDump}\n")
     } catch (IOException ignored) {
 
     }
         
-    
-    describe "service registry names"
-    def registryService = client.registryOperations
 
     def self = currentUser()
-    RegistryPathStatus[] serviceTypes = registryService.list(userPath(self))
-    dumpArray(serviceTypes)
+    List<RegistryPathStatus> serviceTypes = registryService.listFull(homePathForUser(self))
+    dumpCollection(serviceTypes)
 
     def recordsPath = serviceclassPath(self, SliderKeys.APP_TYPE)
 
     Map < String, ServiceRecord > recordMap = RecordOperations.extractServiceRecords(
         registryService,
-        registryService.list(recordsPath))
+        registryService.listFull(recordsPath))
     def serviceRecords = recordMap.values();
     dumpCollection(serviceRecords)
     assert serviceRecords.size() == 1
 
-    def serviceInstance = serviceRecords[0]
-    log.info(serviceInstance.toString())
+    def serviceRecord = serviceRecords[0]
+    log.info(serviceRecord.toString())
 
-    assert 2 <= serviceInstance.external.size()
+    assert 2 <= serviceRecord.external.size()
 
     // hit the registry web page
-
-    def registryEndpoint = serviceInstance.getExternalEndpoint(
+    def registryEndpoint = serviceRecord.getExternalEndpoint(
         CustomRegistryConstants.REGISTRY_REST_API)
     assert registryEndpoint != null
     def registryURL = RegistryTypeUtils.retrieveAddressURLs(registryEndpoint)[0]
-    describe("Registry WADL @ $registryURL")
     
-    def publisherEndpoint = serviceInstance.getExternalEndpoint(
+    describe("Registry WADL @ $registryURL")
+    def publisherEndpoint = serviceRecord.getExternalEndpoint(
         CustomRegistryConstants.PUBLISHER_REST_API)
 
     def publisherURL = RegistryTypeUtils.retrieveAddressURLs(publisherEndpoint)[0]
@@ -231,7 +231,7 @@ class TestStandaloneYarnRegistryAM extends AgentMiniClusterTestBase {
     describe "Registry Retrieval Class"
     // retrieval
 
-    RegistryRetriever retriever = new RegistryRetriever(serviceInstance)
+    RegistryRetriever retriever = new RegistryRetriever(serviceRecord)
     log.info retriever.toString()
     
     assert retriever.hasConfigurations(true)
@@ -286,7 +286,7 @@ class TestStandaloneYarnRegistryAM extends AgentMiniClusterTestBase {
     registryArgs.name = "unknown"
     try {
       client.actionRegistryListYarn(registryArgs)
-    } catch (PathNotFoundException expected) {
+    } catch (UnknownApplicationInstanceException expected) {
       // expected 
     }
 
@@ -296,7 +296,7 @@ class TestStandaloneYarnRegistryAM extends AgentMiniClusterTestBase {
     registryArgs.serviceType = "org-apache-hadoop"
     try {
       client.actionRegistryListYarn(registryArgs)
-    } catch (PathNotFoundException expected) {
+    } catch (UnknownApplicationInstanceException expected) {
       // expected 
     }
 
@@ -311,7 +311,7 @@ class TestStandaloneYarnRegistryAM extends AgentMiniClusterTestBase {
     describe registryArgs.toString()
 
     def listedInstance = client.actionRegistryListYarn(registryArgs)
-    assert listedInstance[0].id == serviceInstance.id
+    assert listedInstance[0].id == serviceRecord.id
     
 
     // listconf 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy
index 7c5de08..07fa327 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy
@@ -33,7 +33,6 @@ import org.apache.slider.core.conf.MapOperations
 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.registry.info.ServiceInstanceData
 import org.apache.slider.providers.ProviderRole
 import org.apache.slider.providers.ProviderService
 import org.apache.slider.server.appmaster.actions.QueueAccess
@@ -46,7 +45,6 @@ import org.apache.slider.server.appmaster.web.rest.agent.HeartBeatResponse
 import org.apache.slider.server.appmaster.web.rest.agent.Register
 import org.apache.slider.server.appmaster.web.rest.agent.RegistrationResponse
 import org.apache.slider.server.appmaster.web.rest.agent.RegistrationStatus
-import org.apache.slider.server.services.registry.RegistryViewForProviders
 import org.apache.slider.providers.ProviderCompleted
 import org.apache.hadoop.service.Service.STATE
 import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders
@@ -208,7 +206,6 @@ class MockProviderService implements ProviderService {
   @Override
   void bind(
       StateAccessForProviders stateAccessor,
-      RegistryViewForProviders registry,
       QueueAccess queueAccess,
       List<Container> liveContainers) {
 
@@ -248,7 +245,6 @@ class MockProviderService implements ProviderService {
       URL amWebURI,
       URL agentOpsURI,
       URL agentStatusURI,
-      ServiceInstanceData registryInstanceData,
       ServiceRecord serviceRecord) throws IOException {
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
index 1c98e94..c49abec 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
@@ -52,7 +52,12 @@ public class TestClusterSpecificationBlock {
         appState)
     ProviderService providerService = new MockProviderService();
 
-    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, providerAppState, providerService, null);
+    WebAppApiImpl inst = new WebAppApiImpl(
+        clusterProto,
+        providerAppState,
+        providerService,
+        null,
+        null);
 
     Injector injector = Guice.createInjector(new AbstractModule() {
           @Override

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
index 2bfd279..9292285 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
@@ -59,7 +59,12 @@ public class TestContainerStatsBlock extends BaseMockAppStateTest {
         "undefined",
         appState)
 
-    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, providerAppState, providerService, null);
+    WebAppApiImpl inst = new WebAppApiImpl(
+        clusterProto,
+        providerAppState,
+        providerService,
+        null,
+        null);
 
     Injector injector = Guice.createInjector(new AbstractModule() {
           @Override

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
index 86d3ef4..a3a4118 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
@@ -27,7 +27,6 @@ import org.apache.hadoop.yarn.webapp.hamlet.Hamlet
 import org.apache.slider.api.SliderClusterProtocol
 import org.apache.slider.providers.ProviderService
 import org.apache.slider.server.appmaster.model.mock.*
-import org.apache.slider.server.appmaster.state.AppState
 import org.apache.slider.server.appmaster.state.ProviderAppState
 import org.apache.slider.server.appmaster.web.WebAppApi
 import org.apache.slider.server.appmaster.web.WebAppApiImpl
@@ -52,7 +51,12 @@ public class TestIndexBlock extends BaseMockAppStateTest {
         "undefined",
         appState)
 
-    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, providerAppState, providerService, null);
+    WebAppApiImpl inst = new WebAppApiImpl(
+        clusterProto,
+        providerAppState,
+        providerService,
+        null,
+        null);
 
     Injector injector = Guice.createInjector(new AbstractModule() {
           @Override

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
index 7e8fec9..7621a4d 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.Path
 import org.apache.hadoop.util.Shell
 import org.apache.hadoop.yarn.api.records.ApplicationReport
 import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.registry.client.types.ServiceRecord
 import org.apache.slider.api.ClusterDescription
 import org.apache.slider.api.ClusterNode
 import org.apache.slider.api.RoleKeys
@@ -689,19 +690,15 @@ class SliderTestUtils extends Assert {
   }
 
   public static void dumpRegistryInstances(
-      List<CuratorServiceInstance<ServiceInstanceData>> instances) {
+      Map<String, ServiceRecord> instances) {
     describe "service registry slider instances"
-    JsonSerDeser<ServiceInstanceData> serDeser = new JsonSerDeser<>(
-        ServiceInstanceData)
-
-    instances.each { CuratorServiceInstance<ServiceInstanceData> svc ->
-      ServiceInstanceData payload = svc.payload
-      def json = serDeser.toJson(payload)
-      log.info("service $svc payload=\n$json")
+    instances.each { Map.Entry<String, ServiceRecord> it ->
+      log.info(" $it.key : $it.value")
     }
     describe "end list service registry slider instances"
   }
 
+  
   public static void dumpRegistryInstanceIDs(List<String> instanceIds) {
     describe "service registry instance IDs"
     dumpCollection(instanceIds)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
index c825fb8..452b03b 100644
--- a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
+++ b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
@@ -54,7 +54,6 @@ import org.slf4j.LoggerFactory;
 
 import javax.ws.rs.core.MediaType;
 import java.io.File;
-import java.io.IOException;
 import java.net.URI;
 //import java.nio.file.FileVisitResult;
 //import java.nio.file.Files;
@@ -139,7 +138,7 @@ public class TestAMAgentWebServices {
                                                              appState);
 
     slider = new WebAppApiImpl(new MockSliderClusterProtocol(), providerAppState,
-                               new MockProviderService(), null);
+                               new MockProviderService(), null, null);
 
     MapOperations compOperations = new MapOperations();
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
index 91aa2b0..20889ac 100644
--- a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
+++ b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
@@ -191,7 +191,7 @@ public class TestAMManagementWebServices extends JerseyTest {
             appState);
 
         slider = new WebAppApiImpl(new MockSliderClusterProtocol(), providerAppState,
-                                   new MockProviderService(), null);
+                                   new MockProviderService(), null, null);
 
         bind(SliderJacksonJaxbJsonProvider.class);
         bind(MockSliderAMWebServices.class);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java
index 0b1d0fb..7d09ee6 100644
--- a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java
+++ b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java
@@ -20,8 +20,6 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.slider.providers.agent.AgentProviderService;
 import org.apache.slider.server.appmaster.actions.QueueAccess;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
-import org.apache.slider.server.services.registry.RegistryViewForProviders;
-import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,10 +41,9 @@ public class TestAgentProviderService extends AgentProviderService {
 
   @Override
   public void bind(StateAccessForProviders stateAccessor,
-      RegistryViewForProviders reg,
       QueueAccess queueAccess,
       List<Container> liveContainers) {
-    super.bind(stateAccessor, reg, queueAccess, liveContainers);
+    super.bind(stateAccessor, queueAccess, liveContainers);
     Map<String,String> dummyProps = new HashMap<String, String>();
     dummyProps.put("prop1", "val1");
     dummyProps.put("prop2", "val2");

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestHBaseMaster.groovy
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestHBaseMaster.groovy b/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestHBaseMaster.groovy
index 634ebe2..9c53d2c 100644
--- a/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestHBaseMaster.groovy
+++ b/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestHBaseMaster.groovy
@@ -20,11 +20,12 @@ package org.apache.slider.providers.hbase.minicluster.live
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.hadoop.yarn.registry.client.binding.RegistryOperationUtils
+import org.apache.hadoop.yarn.registry.client.types.ServiceRecord
 import org.apache.slider.common.SliderXmlConfKeys
 import org.apache.slider.api.ClusterDescription
 import org.apache.slider.api.RoleKeys
 import org.apache.slider.core.registry.docstore.PublishedConfigSet
-import org.apache.slider.core.registry.info.ServiceInstanceData
 import org.apache.slider.core.registry.retrieve.RegistryRetriever
 import org.apache.slider.providers.hbase.HBaseKeys
 import org.apache.slider.core.zk.ZKIntegration
@@ -32,8 +33,6 @@ import org.apache.slider.common.params.Arguments
 import org.apache.slider.client.SliderClient
 import org.apache.slider.providers.hbase.minicluster.HBaseMiniClusterTestBase
 import org.apache.slider.core.main.ServiceLauncher
-import org.apache.slider.server.services.curator.CuratorServiceInstance
-import org.apache.slider.server.services.registry.SliderRegistryService
 import org.junit.Test
 
 /**
@@ -77,28 +76,24 @@ class TestHBaseMaster extends HBaseMiniClusterTestBase {
     
     // look up the registry entries for HBase 
     describe "service registry names"
-    SliderRegistryService registryService = client.registry
-    def names = registryService.getServiceTypes();
+    Map<String, ServiceRecord> records = RegistryOperationUtils.listServiceRecords(
+        client.registryOperations,
+        RegistryOperationUtils.serviceclassPath(
+            RegistryOperationUtils.homePathForCurrentUser(),
+            HBaseKeys.HBASE_SERVICE_TYPE
+        )
+    )
+
+    def names = records.keySet()
     dumpRegistryServiceTypes(names)
 
-    List<CuratorServiceInstance<ServiceInstanceData>> instances =
-        client.listRegistryInstances();
 
-    def hbaseInstances = registryService.findInstances( HBaseKeys.HBASE_SERVICE_TYPE, null)
+
+    def hbaseInstances = records.values()
     assert hbaseInstances.size() == 1
-    def hbaseService = hbaseInstances[0]
-    assert hbaseService
-    def hbaseServiceData = hbaseService.payload
+    ServiceRecord hbaseServiceData = hbaseInstances[0]
     log.info "HBase service 0 == $hbaseServiceData"
     assert hbaseServiceData.id 
-    assert hbaseServiceData.serviceType == HBaseKeys.HBASE_SERVICE_TYPE
-
-    hbaseInstances = registryService.findInstances(
-        HBaseKeys.HBASE_SERVICE_TYPE,
-        clustername)
-    assert hbaseInstances.size() == 1
-    def hbaseServiceData2 = hbaseInstances[0].payload
-    assert hbaseServiceData == hbaseServiceData2
 
     RegistryRetriever retriever = new RegistryRetriever(hbaseServiceData)
     log.info retriever.toString()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/44e61659/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestTwoLiveClusters.groovy
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestTwoLiveClusters.groovy b/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestTwoLiveClusters.groovy
index f7bb8d2..a415524 100644
--- a/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestTwoLiveClusters.groovy
+++ b/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestTwoLiveClusters.groovy
@@ -20,14 +20,16 @@ package org.apache.slider.providers.hbase.minicluster.live
 
 import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
+import org.apache.hadoop.yarn.registry.client.api.RegistryConstants
+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.types.ServiceRecord
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.client.SliderClient
 import org.apache.slider.providers.hbase.HBaseKeys
 import org.apache.slider.providers.hbase.minicluster.HBaseMiniClusterTestBase
 import org.apache.slider.core.main.ServiceLauncher
-import org.apache.slider.core.registry.info.ServiceInstanceData
-import org.apache.slider.server.services.curator.CuratorServiceInstance
-import org.apache.slider.server.services.registry.SliderRegistryService
 import org.junit.Test
 
 @CompileStatic
@@ -83,10 +85,15 @@ class TestTwoLiveClusters extends HBaseMiniClusterTestBase {
 
     // registry instances    def names = client.listRegistryNames(clustername)
     describe "service registry names"
-    SliderRegistryService registry = cluster2Client.registry
-    def names = registry.getServiceTypes();
-    dumpRegistryServiceTypes(names)
+    RegistryOperations registry = cluster2Client.registryOperations
+    def home = RegistryOperationUtils.homePathForCurrentUser()
+    def names = RegistryOperationUtils.listServiceRecords(registry,
+        RegistryPathUtils.join(home, RegistryConstants.PATH_USER_SERVICES))
 
+    def stats = registry.listFull(
+        RegistryPathUtils.join(home, RegistryConstants.PATH_USER_SERVICES))
+    
+    dumpCollection(stats)
     List<String> instanceIds = sliderClient.listRegisteredSliderInstances()
 
 
@@ -95,18 +102,20 @@ class TestTwoLiveClusters extends HBaseMiniClusterTestBase {
     assert instanceIds.size() == 2
 
 
-    List<CuratorServiceInstance<ServiceInstanceData>> instances =
+    Map<String, ServiceRecord> instances =
         sliderClient.listRegistryInstances()
     dumpRegistryInstances(instances)
     assert instances.size() == 2
 
-    def hbaseInstances = registry.findInstances(
-        HBaseKeys.HBASE_SERVICE_TYPE, null)
+    Map<String, ServiceRecord> hbaseInstances =
+        RegistryOperationUtils.listServiceRecords(registry,
+        RegistryOperationUtils.serviceclassPath(
+            RegistryOperationUtils.currentUser(),
+            HBaseKeys.HBASE_SERVICE_TYPE));
+        
     assert hbaseInstances.size() == 2
-    def hbase1ServiceData = registry.findInstance(
-        HBaseKeys.HBASE_SERVICE_TYPE, clustername1).payload
-    def hbase2ServiceData = registry.findInstance(
-        HBaseKeys.HBASE_SERVICE_TYPE, clustername2).payload
+    def hbase1ServiceData = hbaseInstances[clustername1]
+    def hbase2ServiceData = hbaseInstances[clustername2]
     assert !(hbase1ServiceData == hbase2ServiceData)
 
     clusterActionFreeze(cluster2Client, clustername2, "stop cluster 2")