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/08 00:17:48 UTC

[2/5] git commit: SLIDER-149 in sync with latest YARN-913 changes

SLIDER-149 in sync with latest YARN-913 changes


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

Branch: refs/heads/feature/SLIDER-149_Support_a_YARN_service_registry
Commit: 27381a6dc29589a06d98be43901f175230d54198
Parents: d084e9e
Author: Steve Loughran <st...@apache.org>
Authored: Mon Oct 6 21:20:02 2014 -0700
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Oct 6 21:20:02 2014 -0700

----------------------------------------------------------------------
 .../funtest/accumulo/AccumuloBasicIT.groovy     | 10 ++-
 .../org/apache/slider/client/SliderClient.java  | 64 ++++++++++++--------
 .../server/appmaster/SliderAppMaster.java       | 18 +++---
 .../YarnRegistryViewForProviders.java           |  8 +--
 .../TestStandaloneYarnRegistryAM.groovy         | 19 +++---
 .../registry/TestRegistryRestMarshalling.groovy |  5 +-
 .../registry/TestRegistryRestResources.groovy   |  8 ++-
 .../apache/slider/test/MicroZKCluster.groovy    |  2 +-
 .../providers/hbase/HBaseProviderService.java   |  9 ++-
 .../minicluster/live/TestHBaseMaster.groovy     |  3 +-
 10 files changed, 86 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/27381a6d/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 fd8af9a..2f6674e 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
@@ -30,7 +30,6 @@ import org.apache.slider.accumulo.CustomAuthenticator
 import org.apache.slider.api.ClusterDescription
 import org.apache.slider.client.SliderClient
 import org.apache.slider.common.SliderKeys
-import org.apache.slider.common.tools.SliderUtils
 import org.apache.slider.core.conf.ConfTree
 import org.apache.slider.core.persist.ConfTreeSerDeser
 import org.apache.slider.core.registry.docstore.PublishedConfiguration
@@ -40,6 +39,9 @@ import org.apache.slider.funtest.framework.SliderShell
 import org.junit.Before
 import org.junit.Test
 
+import static org.apache.hadoop.yarn.registry.client.binding.RegistryUtils.currentUser
+import static org.apache.hadoop.yarn.registry.client.binding.RegistryUtils.servicePath
+
 @Slf4j
 class AccumuloBasicIT extends AccumuloAgentCommandTestBase {
   protected static final String PROVIDER_PROPERTY = "site.accumulo-site." +
@@ -170,8 +172,10 @@ class AccumuloBasicIT extends AccumuloAgentCommandTestBase {
     Exception caught;
     while (true) {
       try {
-        ServiceRecord instance =
-          sliderClient.lookupServiceRecord(SliderKeys.APP_TYPE, clusterName)
+        String path = servicePath(currentUser(),
+            SliderKeys.APP_TYPE,
+            clusterName);
+        ServiceRecord instance = sliderClient.resolve(path)
         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/27381a6d/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 a5bbccc..602d6cf 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
@@ -48,7 +48,6 @@ import static org.apache.hadoop.yarn.registry.client.binding.RegistryUtils.*;
 import org.apache.hadoop.yarn.registry.client.binding.RegistryUtils;
 import org.apache.hadoop.yarn.registry.client.exceptions.NoRecordException;
 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;
 import org.apache.slider.api.ClusterDescription;
 import org.apache.slider.api.ClusterNode;
@@ -127,15 +126,11 @@ import org.apache.slider.server.services.utility.AbstractSliderLaunchedService;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
-import org.codehaus.jackson.map.DeserializationConfig;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.SerializationConfig;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileWriter;
@@ -2402,15 +2397,19 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     // the arguments
     args.validate();
     RegistryOperations operations = getRegistryOperations();
-    String serviceclassPath = args.path;
+    String path = args.path;
     Collection<ServiceRecord> serviceRecords;
     try {
       if (args.list) {
-        actionRegistryListYarn(args);
+        Map<String, ServiceRecord> recordMap =
+            listServiceRecords(operations, path);
+        serviceRecords = recordMap.values();
+        // list records out
       } else  {
-        ServiceRecord instance = lookupServiceRecord(registryArgs);
+        ServiceRecord instance = resolve(path);
         serviceRecords = new ArrayList<ServiceRecord>(1);
         serviceRecords.add(instance);
+        // list or save records
       }
 //      JDK7
     } catch (FileNotFoundException e) {
@@ -2442,14 +2441,14 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     registryArgs.validate();
     try {
       if (registryArgs.list) {
-        actionRegistryListYarn(registryArgs);
+        actionRegistryList(registryArgs);
       } else if (registryArgs.listConf) {
         // list the configurations
         actionRegistryListConfigsYarn(registryArgs);
       } else if (SliderUtils.isSet(registryArgs.getConf)) {
         // get a configuration
         PublishedConfiguration publishedConfiguration =
-            actionRegistryGetConfigYarn(registryArgs);
+            actionRegistryGetConfig(registryArgs);
         outputConfig(publishedConfiguration, registryArgs);
       } else {
         // it's an unknown command
@@ -2478,7 +2477,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    * @throws IOException Network or other problems
    */
   @VisibleForTesting
-  public Collection<ServiceRecord> actionRegistryListYarn(
+  public Collection<ServiceRecord> actionRegistryList(
       ActionRegistryArgs registryArgs)
       throws YarnException, IOException {
     String serviceType = registryArgs.serviceType;
@@ -2486,23 +2485,21 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     RegistryOperations operations = getRegistryOperations();
     Collection<ServiceRecord> serviceRecords;
     if (StringUtils.isEmpty(name)) {
-      String serviceclassPath =
+      String path =
           serviceclassPath(
               currentUser(),
               serviceType);
 
       try {
         Map<String, ServiceRecord> recordMap =
-            listServiceRecords(operations, serviceclassPath);
-        RegistryPathStatus[] listDir;
+            listServiceRecords(operations, path);
         if (recordMap.isEmpty()) {
           throw new UnknownApplicationInstanceException(
-              "No applications registered under " + serviceclassPath);
+              "No applications registered under " + path);
         }
         serviceRecords = recordMap.values();
       } catch (PathNotFoundException e) {
-        throw new UnknownApplicationInstanceException(e.getPath().toString(),
-            e);
+        throw new UnknownApplicationInstanceException(path, e);
       }
     } else {
       ServiceRecord instance = lookupServiceRecord(registryArgs);
@@ -2792,9 +2789,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   private void logInstance(ServiceRecord instance,
       boolean verbose) {
     if (!verbose) {
-      log.info("{}", instance.yarn_id);
+      log.info("{}", instance.getYarn_id());
     } else {
-      log.info("{}: ", instance.yarn_id);
+      log.info("{}: ", instance.getYarn_id());
       logEndpoints(instance);
     }
   }
@@ -2848,7 +2845,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    * @throws FileNotFoundException if the config is not found
    */
   @VisibleForTesting
-  public PublishedConfiguration actionRegistryGetConfigYarn(ActionRegistryArgs registryArgs)
+  public PublishedConfiguration actionRegistryGetConfig(ActionRegistryArgs registryArgs)
       throws YarnException, IOException {
     ServiceRecord instance = lookupServiceRecord(registryArgs);
 
@@ -2910,7 +2907,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   private ServiceRecord lookupServiceRecord(ActionRegistryArgs registryArgs) throws
       SliderException,
       IOException {
-    return lookupServiceRecord(registryArgs.serviceType, registryArgs.name);
+    String path = servicePath(currentUser(), registryArgs.serviceType,
+        registryArgs.name);
+    return resolve(path);
   }
 
   /**
@@ -2925,18 +2924,33 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    */
   public ServiceRecord lookupServiceRecord(String serviceType, String id)
       throws IOException, SliderException {
+    String path = servicePath(currentUser(), serviceType, id);
+    return resolve(path);
+  }
+
+  /**
+   * 
+   * Look up an instance
+   * @param path path
+   * @return instance data
+   * @throws UnknownApplicationInstanceException no path or service record
+   * at the end of the path
+   * @throws SliderException other failures
+   * @throws IOException IO problems or wrapped exceptions
+   */
+  public ServiceRecord resolve(String path)
+      throws IOException, SliderException {
     try {
       return getRegistryOperations().resolve(
-          servicePath(currentUser(),
-              serviceType, id));
+          path);
       // TODO JDK7 SWITCH
     } catch (PathNotFoundException e) {
       throw new UnknownApplicationInstanceException(e.getPath().toString(), e);
     } catch (NoRecordException e) {
       throw new UnknownApplicationInstanceException(e.getPath().toString(), e);
     }
-  } 
-  
+  }
+
   /**
    * List instances in the registry for the current user
    * @return a list of slider registry instances

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/27381a6d/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 a259e42..6f7275c 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
@@ -56,7 +56,7 @@ 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.types.PersistencePolicies;
+import org.apache.hadoop.yarn.registry.client.types.yarn.PersistencePolicies;
 import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
 import org.apache.hadoop.yarn.registry.client.binding.RegistryTypeUtils;
 import org.apache.hadoop.yarn.registry.server.integration.RMRegistryOperationsService;
@@ -931,8 +931,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
     // Yarn registry
     ServiceRecord serviceRecord = new ServiceRecord();
-    serviceRecord.yarn_id = appid.toString();
-    serviceRecord.yarn_persistence = PersistencePolicies.APPLICATION;
+    serviceRecord.putYarn_id(appid.toString());
+    serviceRecord.putYarn_persistence(PersistencePolicies.APPLICATION);
     serviceRecord.description = "Slider Application Master";
 
     serviceRecord.addExternalEndpoint(
@@ -964,8 +964,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     // and a shorter lived binding to the app
     String attempt = appAttemptID.toString();
     ServiceRecord attemptRecord = new ServiceRecord(serviceRecord);
-    attemptRecord.yarn_id = attempt;
-    attemptRecord.yarn_persistence = PersistencePolicies.APPLICATION_ATTEMPT;
+    attemptRecord.putYarn_id(attempt);
+    attemptRecord.putYarn_persistence(PersistencePolicies.APPLICATION_ATTEMPT);
     yarnRegistryOperations.putComponent(
         RegistryPathUtils.encodeYarnID(attempt),
         serviceRecord);
@@ -1007,10 +1007,10 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     // this is where component registrations will go
     log.info("Registering component {}", id);
     String cid = RegistryPathUtils.encodeYarnID(id.toString());
-    ServiceRecord container = new ServiceRecord(
-        cid,
-        description,
-        PersistencePolicies.CONTAINER, null);
+    ServiceRecord container = new ServiceRecord();
+    container.putYarn_id(cid);
+    container.description = description;
+    container.putYarn_persistence(PersistencePolicies.CONTAINER);
     try {
       yarnRegistryOperations.putComponent(cid, container);
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/27381a6d/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 d0c891d..10ca85c 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
@@ -19,11 +19,11 @@
 package org.apache.slider.server.services.yarnregistry;
 
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.registry.client.api.BindFlags;
 import org.apache.hadoop.yarn.registry.client.api.RegistryOperations;
 import org.apache.hadoop.yarn.registry.client.binding.RegistryUtils;
 import org.apache.hadoop.yarn.registry.client.binding.RegistryPathUtils;
 
-import org.apache.hadoop.yarn.registry.client.api.CreateFlags;
 import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
 
 import java.io.IOException;
@@ -107,7 +107,7 @@ public class YarnRegistryViewForProviders {
     String path = RegistryUtils.componentPath(
         user, serviceClass, serviceName, componentName);
     registryOperations.mknode(RegistryPathUtils.parentOf(path), true);
-    registryOperations.create(path, record, CreateFlags.OVERWRITE);
+    registryOperations.bind(path, record, BindFlags.OVERWRITE);
   }
 
   /**
@@ -125,7 +125,7 @@ public class YarnRegistryViewForProviders {
     String path = RegistryUtils.servicePath(
         username, serviceClass, serviceName);
     registryOperations.mknode(RegistryPathUtils.parentOf(path), true);
-    registryOperations.create(path, record, CreateFlags.OVERWRITE);
+    registryOperations.bind(path, record, BindFlags.OVERWRITE);
   }
 
   /**
@@ -142,7 +142,7 @@ public class YarnRegistryViewForProviders {
     String path = RegistryUtils.servicePath(
         user, serviceClass, serviceName);
     registryOperations.mknode(RegistryPathUtils.parentOf(path), true);
-    registryOperations.create(path, record, CreateFlags.OVERWRITE);
+    registryOperations.bind(path, record, BindFlags.OVERWRITE);
   }
 
   public void rmComponent(String componentName) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/27381a6d/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 b9238f4..7e74625 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
@@ -25,9 +25,10 @@ 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.RegistryTypeUtils
-import org.apache.hadoop.yarn.registry.client.services.RegistryOperationsClient
+import org.apache.hadoop.yarn.registry.client.impl.RegistryOperationsClient
 import org.apache.hadoop.yarn.registry.client.types.RegistryPathStatus
 import org.apache.hadoop.yarn.registry.client.types.ServiceRecord
+import org.apache.hadoop.yarn.registry.client.types.yarn.YarnRegistryAttributes
 import org.apache.slider.core.exceptions.UnknownApplicationInstanceException
 
 import static org.apache.hadoop.yarn.registry.client.binding.RegistryUtils.*
@@ -158,7 +159,8 @@ class TestStandaloneYarnRegistryAM extends AgentMiniClusterTestBase {
     def serviceRecord = serviceRecords[0]
     log.info(serviceRecord.toString())
 
-    assert serviceRecord.yarn_id != null;
+    assert serviceRecord[YarnRegistryAttributes.YARN_ID] != null
+    assert serviceRecord[YarnRegistryAttributes.YARN_PERSISTENCE] != ""
     def externalEndpoints = serviceRecord.external;
     assert externalEndpoints.size() > 0
 
@@ -299,7 +301,7 @@ class TestStandaloneYarnRegistryAM extends AgentMiniClusterTestBase {
     registryArgs.list = true;
     registryArgs.name = "unknown"
     try {
-      client.actionRegistryListYarn(registryArgs)
+      client.actionRegistryList(registryArgs)
     } catch (UnknownApplicationInstanceException expected) {
       // expected 
     }
@@ -309,7 +311,7 @@ class TestStandaloneYarnRegistryAM extends AgentMiniClusterTestBase {
     registryArgs.name = null
     registryArgs.serviceType = "org-apache-hadoop"
     try {
-      client.actionRegistryListYarn(registryArgs)
+      client.actionRegistryList(registryArgs)
     } catch (UnknownApplicationInstanceException expected) {
       // expected 
     }
@@ -324,9 +326,12 @@ class TestStandaloneYarnRegistryAM extends AgentMiniClusterTestBase {
     //now expect list to work
     describe registryArgs.toString()
 
-    def listedInstance = client.actionRegistryListYarn(registryArgs)
-    assert listedInstance[0].yarn_id == serviceRecord.yarn_id
-    
+    def listedInstance = client.actionRegistryList(registryArgs)
+
+    def resolvedRecord = listedInstance[0]
+    assert resolvedRecord[YarnRegistryAttributes.YARN_ID] == serviceRecord.getYarn_id()
+    assert resolvedRecord[YarnRegistryAttributes.YARN_PERSISTENCE] == serviceRecord.getYarn_persistence()
+   
 
     // listconf 
     registryArgs.list = false;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/27381a6d/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/rest/registry/TestRegistryRestMarshalling.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/rest/registry/TestRegistryRestMarshalling.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/rest/registry/TestRegistryRestMarshalling.groovy
index 44c8f05..d67dfeb 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/rest/registry/TestRegistryRestMarshalling.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/rest/registry/TestRegistryRestMarshalling.groovy
@@ -19,6 +19,7 @@
 package org.apache.slider.server.appmaster.web.rest.registry
 
 import groovy.transform.CompileStatic
+import org.apache.hadoop.yarn.registry.client.types.yarn.YarnRegistryAttributes
 import org.junit.Test
 
 /**
@@ -39,8 +40,8 @@ class TestRegistryRestMarshalling {
 
     def serviceRecord = unmarshalled.service
     assert serviceRecord
-    assert serviceRecord.yarn_id !=null
-    assert serviceRecord.yarn_persistence != 0
+    assert serviceRecord[YarnRegistryAttributes.YARN_ID] != null
+    assert serviceRecord[YarnRegistryAttributes.YARN_PERSISTENCE] != ""
   }
 
   

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/27381a6d/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/rest/registry/TestRegistryRestResources.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/rest/registry/TestRegistryRestResources.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/rest/registry/TestRegistryRestResources.groovy
index e6d31fd..ee678f9 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/rest/registry/TestRegistryRestResources.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/rest/registry/TestRegistryRestResources.groovy
@@ -26,6 +26,7 @@ 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.binding.RegistryUtils
+import org.apache.hadoop.yarn.registry.client.types.yarn.YarnRegistryAttributes
 import org.apache.slider.api.StatusKeys
 import org.apache.slider.client.SliderClient
 import org.apache.slider.common.SliderKeys
@@ -157,8 +158,9 @@ class TestRegistryRestResources extends AgentTestBase {
     def unmarshalled = pem.fromJson(responseStr)
     def r1 = unmarshalled.service
     assert r1
-    assert r1.yarn_id != null
-    assert r1.yarn_persistence != 0
+    assert r1[YarnRegistryAttributes.YARN_ID] != null
+    assert r1[YarnRegistryAttributes.YARN_PERSISTENCE] != ""
+
 
     // and via the web resource AP
     jsonBuilder = webResource.type(MediaType.APPLICATION_JSON);
@@ -166,7 +168,7 @@ class TestRegistryRestResources extends AgentTestBase {
 
     def serviceRecord = entryResource.service
     assert serviceRecord != null;
-    assert serviceRecord.yarn_id != null;
+    assert serviceRecord[YarnRegistryAttributes.YARN_ID] != null
     def externalEndpoints = serviceRecord.external;
     assert externalEndpoints.size() > 0
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/27381a6d/slider-core/src/test/groovy/org/apache/slider/test/MicroZKCluster.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/MicroZKCluster.groovy b/slider-core/src/test/groovy/org/apache/slider/test/MicroZKCluster.groovy
index 7f72490..c2d6a32 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/MicroZKCluster.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/MicroZKCluster.groovy
@@ -22,7 +22,7 @@ import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.yarn.registry.client.api.RegistryOperations
-import org.apache.hadoop.yarn.registry.client.services.RegistryOperationsService
+import org.apache.hadoop.yarn.registry.client.impl.zk.RegistryOperationsService
 import org.apache.hadoop.yarn.registry.server.services.MicroZookeeperService
 import org.apache.slider.common.tools.SliderUtils
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/27381a6d/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java b/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java
index 24abdba..ae9084b 100644
--- a/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java
+++ b/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.registry.client.binding.RegistryTypeUtils;
-import org.apache.hadoop.yarn.registry.client.types.PersistencePolicies;
+import org.apache.hadoop.yarn.registry.client.types.yarn.PersistencePolicies;
 import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
 import org.apache.slider.api.InternalKeys;
 import org.apache.slider.common.SliderKeys;
@@ -40,7 +40,6 @@ 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.providers.AbstractProviderService;
-import org.apache.slider.providers.ProviderCompleted;
 import org.apache.slider.providers.ProviderCore;
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.providers.ProviderUtils;
@@ -258,9 +257,9 @@ public class HBaseProviderService extends AbstractProviderService
     String name = amState.getApplicationName() ;
     ServiceRecord serviceRecord = new ServiceRecord();
     // bond lifespan to the application
-    serviceRecord.yarn_id  = yarnRegistry.getApplicationAttemptId()
-                                         .getApplicationId().toString();
-    serviceRecord.yarn_persistence = PersistencePolicies.APPLICATION;
+    serviceRecord.putYarn_id(yarnRegistry.getApplicationAttemptId()
+                                         .getApplicationId().toString());
+    serviceRecord.putYarn_persistence(PersistencePolicies.APPLICATION);
     try {
       URL configURL = new URL(amWebAPI,
           SLIDER_PATH_PUBLISHER + "/" + HBASE_SERVICE_TYPE);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/27381a6d/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 84db30f..b73432a 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
@@ -22,6 +22,7 @@ import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.yarn.registry.client.binding.RegistryUtils
 import org.apache.hadoop.yarn.registry.client.types.ServiceRecord
+import org.apache.hadoop.yarn.registry.client.types.yarn.YarnRegistryAttributes
 import org.apache.slider.common.SliderXmlConfKeys
 import org.apache.slider.api.ClusterDescription
 import org.apache.slider.api.RoleKeys
@@ -93,7 +94,7 @@ class TestHBaseMaster extends HBaseMiniClusterTestBase {
     assert hbaseInstances.size() == 1
     ServiceRecord hbaseServiceData = hbaseInstances[0]
     log.info "HBase service 0 == $hbaseServiceData"
-    assert hbaseServiceData.yarn_id 
+    assert hbaseServiceData[YarnRegistryAttributes.YARN_ID] 
 
     RegistryRetriever retriever = new RegistryRetriever(hbaseServiceData)
     log.info retriever.toString()