You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2014/10/08 21:54:48 UTC

[1/6] YARN-913 service registry: YARN-2652 add hadoop-yarn-registry package under hadoop-yarn

Repository: hadoop
Updated Branches:
  refs/heads/branch-2.6 7ec65c861 -> 088ae9c5b


http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureRMRegistryOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureRMRegistryOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureRMRegistryOperations.java
new file mode 100644
index 0000000..bb95340
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureRMRegistryOperations.java
@@ -0,0 +1,350 @@
+/*
+ * 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.hadoop.registry.secure;
+
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.PathPermissionException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.ServiceStateException;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.api.RegistryOperationsFactory;
+import org.apache.hadoop.registry.client.exceptions.NoPathPermissionsException;
+import org.apache.hadoop.registry.client.impl.zk.ZKPathDumper;
+import org.apache.hadoop.registry.client.impl.RegistryOperationsClient;
+import org.apache.hadoop.registry.client.impl.zk.RegistrySecurity;
+import org.apache.hadoop.registry.client.impl.zk.ZookeeperConfigOptions;
+import org.apache.hadoop.registry.server.integration.RMRegistryOperationsService;
+import org.apache.hadoop.registry.server.services.RegistryAdminService;
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.login.LoginException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+
+import static org.apache.hadoop.registry.client.api.RegistryConstants.*;
+
+/**
+ * Verify that the {@link RMRegistryOperationsService} works securely
+ */
+public class TestSecureRMRegistryOperations extends AbstractSecureRegistryTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestSecureRMRegistryOperations.class);
+  private Configuration secureConf;
+  private Configuration zkClientConf;
+  private UserGroupInformation zookeeperUGI;
+
+  @Before
+  public void setupTestSecureRMRegistryOperations() throws Exception {
+    startSecureZK();
+    secureConf = new Configuration();
+    secureConf.setBoolean(KEY_REGISTRY_SECURE, true);
+
+    // create client conf containing the ZK quorum
+    zkClientConf = new Configuration(secureZK.getConfig());
+    zkClientConf.setBoolean(KEY_REGISTRY_SECURE, true);
+    assertNotEmpty(zkClientConf.get(RegistryConstants.KEY_REGISTRY_ZK_QUORUM));
+
+    // ZK is in charge
+    secureConf.set(KEY_REGISTRY_SYSTEM_ACCOUNTS, "sasl:zookeeper@");
+    zookeeperUGI = loginUGI(ZOOKEEPER, keytab_zk);
+  }
+
+  @After
+  public void teardownTestSecureRMRegistryOperations() {
+  }
+
+  /**
+   * Create the RM registry operations as the current user
+   * @return the service
+   * @throws LoginException
+   * @throws FileNotFoundException
+   */
+  public RMRegistryOperationsService startRMRegistryOperations() throws
+      LoginException, IOException, InterruptedException {
+    // kerberos
+    secureConf.set(KEY_REGISTRY_CLIENT_AUTH,
+        REGISTRY_CLIENT_AUTH_KERBEROS);
+    secureConf.set(KEY_REGISTRY_CLIENT_JAAS_CONTEXT, ZOOKEEPER_CLIENT_CONTEXT);
+
+    RMRegistryOperationsService registryOperations = zookeeperUGI.doAs(
+        new PrivilegedExceptionAction<RMRegistryOperationsService>() {
+          @Override
+          public RMRegistryOperationsService run() throws Exception {
+            RMRegistryOperationsService operations
+                = new RMRegistryOperationsService("rmregistry", secureZK);
+            addToTeardown(operations);
+            operations.init(secureConf);
+            LOG.info(operations.bindingDiagnosticDetails());
+            operations.start();
+            return operations;
+          }
+        });
+
+    return registryOperations;
+  }
+
+  /**
+   * test that ZK can write as itself
+   * @throws Throwable
+   */
+  @Test
+  public void testZookeeperCanWriteUnderSystem() throws Throwable {
+
+    RMRegistryOperationsService rmRegistryOperations =
+        startRMRegistryOperations();
+    RegistryOperations operations = rmRegistryOperations;
+    operations.mknode(PATH_SYSTEM_SERVICES + "hdfs",
+        false);
+    ZKPathDumper pathDumper = rmRegistryOperations.dumpPath(true);
+    LOG.info(pathDumper.toString());
+  }
+
+  @Test
+  public void testAnonReadAccess() throws Throwable {
+    RMRegistryOperationsService rmRegistryOperations =
+        startRMRegistryOperations();
+    describe(LOG, "testAnonReadAccess");
+    RegistryOperations operations =
+        RegistryOperationsFactory.createAnonymousInstance(zkClientConf);
+    addToTeardown(operations);
+    operations.start();
+
+    assertFalse("RegistrySecurity.isClientSASLEnabled()==true",
+        RegistrySecurity.isClientSASLEnabled());
+    assertFalse("ZooKeeperSaslClient.isEnabled()==true",
+        ZooKeeperSaslClient.isEnabled());
+    operations.list(PATH_SYSTEM_SERVICES);
+  }
+
+  @Test
+  public void testAnonNoWriteAccess() throws Throwable {
+    RMRegistryOperationsService rmRegistryOperations =
+        startRMRegistryOperations();
+    describe(LOG, "testAnonNoWriteAccess");
+    RegistryOperations operations =
+        RegistryOperationsFactory.createAnonymousInstance(zkClientConf);
+    addToTeardown(operations);
+    operations.start();
+
+    String servicePath = PATH_SYSTEM_SERVICES + "hdfs";
+    expectMkNodeFailure(operations, servicePath);
+  }
+
+  @Test
+  public void testAnonNoWriteAccessOffRoot() throws Throwable {
+    RMRegistryOperationsService rmRegistryOperations =
+        startRMRegistryOperations();
+    describe(LOG, "testAnonNoWriteAccessOffRoot");
+    RegistryOperations operations =
+        RegistryOperationsFactory.createAnonymousInstance(zkClientConf);
+    addToTeardown(operations);
+    operations.start();
+    assertFalse("mknode(/)", operations.mknode("/", false));
+    expectMkNodeFailure(operations, "/sub");
+    expectDeleteFailure(operations, PATH_SYSTEM_SERVICES, true);
+  }
+
+  /**
+   * Expect a mknode operation to fail
+   * @param operations operations instance
+   * @param path path
+   * @throws IOException An IO failure other than those permitted
+   */
+  public void expectMkNodeFailure(RegistryOperations operations,
+      String path) throws IOException {
+    try {
+      operations.mknode(path, false);
+      fail("should have failed to create a node under " + path);
+    } catch (PathPermissionException expected) {
+      // expected
+    } catch (NoPathPermissionsException expected) {
+      // expected
+    }
+  }
+
+  /**
+   * Expect a delete operation to fail
+   * @param operations operations instance
+   * @param path path
+   * @param recursive
+   * @throws IOException An IO failure other than those permitted
+   */
+  public void expectDeleteFailure(RegistryOperations operations,
+      String path, boolean recursive) throws IOException {
+    try {
+      operations.delete(path, recursive);
+      fail("should have failed to delete the node " + path);
+    } catch (PathPermissionException expected) {
+      // expected
+    } catch (NoPathPermissionsException expected) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testAlicePathRestrictedAnonAccess() throws Throwable {
+    RMRegistryOperationsService rmRegistryOperations =
+        startRMRegistryOperations();
+    String aliceHome = rmRegistryOperations.initUserRegistry(ALICE);
+    describe(LOG, "Creating anonymous accessor");
+    RegistryOperations anonOperations =
+        RegistryOperationsFactory.createAnonymousInstance(zkClientConf);
+    addToTeardown(anonOperations);
+    anonOperations.start();
+    anonOperations.list(aliceHome);
+    expectMkNodeFailure(anonOperations, aliceHome + "/anon");
+    expectDeleteFailure(anonOperations, aliceHome, true);
+  }
+
+  @Test
+  public void testUserZookeeperHomePathAccess() throws Throwable {
+    RMRegistryOperationsService rmRegistryOperations =
+        startRMRegistryOperations();
+    final String home = rmRegistryOperations.initUserRegistry(ZOOKEEPER);
+    describe(LOG, "Creating ZK client");
+
+    RegistryOperations operations = zookeeperUGI.doAs(
+        new PrivilegedExceptionAction<RegistryOperations>() {
+          @Override
+          public RegistryOperations run() throws Exception {
+            RegistryOperations operations =
+                RegistryOperationsFactory.createKerberosInstance(zkClientConf,
+                    ZOOKEEPER_CLIENT_CONTEXT);
+            addToTeardown(operations);
+            operations.start();
+
+            return operations;
+          }
+        });
+    operations.list(home);
+    String path = home + "/subpath";
+    operations.mknode(path, false);
+    operations.delete(path, true);
+  }
+
+  @Test
+  public void testUserHomedirsPermissionsRestricted() throws Throwable {
+    // test that the /users/$user permissions are restricted
+    RMRegistryOperationsService rmRegistryOperations =
+        startRMRegistryOperations();
+    // create Alice's dir, so it should have an ACL for Alice
+    final String home = rmRegistryOperations.initUserRegistry(ALICE);
+    List<ACL> acls = rmRegistryOperations.zkGetACLS(home);
+    ACL aliceACL = null;
+    for (ACL acl : acls) {
+      LOG.info(RegistrySecurity.aclToString(acl));
+      Id id = acl.getId();
+      if (id.getScheme().equals(ZookeeperConfigOptions.SCHEME_SASL)
+          && id.getId().startsWith(ALICE)) {
+
+        aliceACL = acl;
+        break;
+      }
+    }
+    assertNotNull(aliceACL);
+    assertEquals(RegistryAdminService.USER_HOMEDIR_ACL_PERMISSIONS,
+        aliceACL.getPerms());
+  }
+
+  @Test
+  public void testDigestAccess() throws Throwable {
+    RMRegistryOperationsService registryAdmin =
+        startRMRegistryOperations();
+    String id = "username";
+    String pass = "password";
+    registryAdmin.addWriteAccessor(id, pass);
+    List<ACL> clientAcls = registryAdmin.getClientAcls();
+    LOG.info("Client ACLS=\n{}", RegistrySecurity.aclsToString(clientAcls));
+
+    String base = "/digested";
+    registryAdmin.mknode(base, false);
+    List<ACL> baseACLs = registryAdmin.zkGetACLS(base);
+    String aclset = RegistrySecurity.aclsToString(baseACLs);
+    LOG.info("Base ACLs=\n{}", aclset);
+    ACL found = null;
+    for (ACL acl : baseACLs) {
+      if (ZookeeperConfigOptions.SCHEME_DIGEST.equals(acl.getId().getScheme())) {
+        found = acl;
+        break;
+      }
+    }
+    assertNotNull("Did not find digest entry in ACLs " + aclset, found);
+    zkClientConf.set(KEY_REGISTRY_USER_ACCOUNTS,
+        "sasl:somebody@EXAMPLE.COM, sasl:other");
+    RegistryOperations operations =
+        RegistryOperationsFactory.createAuthenticatedInstance(zkClientConf,
+            id,
+            pass);
+    addToTeardown(operations);
+    operations.start();
+    RegistryOperationsClient operationsClient =
+        (RegistryOperationsClient) operations;
+    List<ACL> digestClientACLs = operationsClient.getClientAcls();
+    LOG.info("digest client ACLs=\n{}",
+        RegistrySecurity.aclsToString(digestClientACLs));
+    operations.stat(base);
+    operations.mknode(base + "/subdir", false);
+    ZKPathDumper pathDumper = registryAdmin.dumpPath(true);
+    LOG.info(pathDumper.toString());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testNoDigestAuthMissingId() throws Throwable {
+    RegistryOperationsFactory.createAuthenticatedInstance(zkClientConf,
+        "",
+        "pass");
+  }
+
+  @Test(expected = ServiceStateException.class)
+  public void testNoDigestAuthMissingId2() throws Throwable {
+    zkClientConf.set(KEY_REGISTRY_CLIENT_AUTH, REGISTRY_CLIENT_AUTH_DIGEST);
+    zkClientConf.set(KEY_REGISTRY_CLIENT_AUTHENTICATION_ID, "");
+    zkClientConf.set(KEY_REGISTRY_CLIENT_AUTHENTICATION_PASSWORD, "pass");
+    RegistryOperationsFactory.createInstance("DigestRegistryOperations",
+        zkClientConf);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testNoDigestAuthMissingPass() throws Throwable {
+    RegistryOperationsFactory.createAuthenticatedInstance(zkClientConf,
+        "id",
+        "");
+  }
+
+  @Test(expected = ServiceStateException.class)
+  public void testNoDigestAuthMissingPass2() throws Throwable {
+    zkClientConf.set(KEY_REGISTRY_CLIENT_AUTH, REGISTRY_CLIENT_AUTH_DIGEST);
+    zkClientConf.set(KEY_REGISTRY_CLIENT_AUTHENTICATION_ID, "id");
+    zkClientConf.set(KEY_REGISTRY_CLIENT_AUTHENTICATION_PASSWORD, "");
+    RegistryOperationsFactory.createInstance("DigestRegistryOperations",
+        zkClientConf);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureRegistry.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureRegistry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureRegistry.java
new file mode 100644
index 0000000..2dad4bd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureRegistry.java
@@ -0,0 +1,157 @@
+/*
+ * 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.hadoop.registry.secure;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.ServiceOperations;
+import org.apache.hadoop.registry.client.impl.zk.ZKPathDumper;
+import org.apache.hadoop.registry.client.impl.zk.CuratorService;
+import org.apache.hadoop.registry.client.impl.zk.RegistrySecurity;
+import org.apache.zookeeper.CreateMode;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.login.LoginContext;
+
+import static org.apache.hadoop.registry.client.api.RegistryConstants.*;
+
+/**
+ * Verify that the Mini ZK service can be started up securely
+ */
+public class TestSecureRegistry extends AbstractSecureRegistryTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestSecureRegistry.class);
+
+  @Before
+  public void beforeTestSecureZKService() throws Throwable {
+      enableKerberosDebugging();
+  }
+
+  @After
+  public void afterTestSecureZKService() throws Throwable {
+    disableKerberosDebugging();
+    RegistrySecurity.clearZKSaslClientProperties();
+  }
+
+  @Test
+  public void testCreateSecureZK() throws Throwable {
+    startSecureZK();
+    secureZK.stop();
+  }
+
+  @Test
+  public void testInsecureClientToZK() throws Throwable {
+    startSecureZK();
+    userZookeeperToCreateRoot();
+    RegistrySecurity.clearZKSaslClientProperties();
+
+    CuratorService curatorService =
+        startCuratorServiceInstance("insecure client", false);
+
+    curatorService.zkList("/");
+    curatorService.zkMkPath("", CreateMode.PERSISTENT, false,
+        RegistrySecurity.WorldReadWriteACL);
+  }
+
+  /**
+   * test that ZK can write as itself
+   * @throws Throwable
+   */
+  @Test
+  public void testZookeeperCanWrite() throws Throwable {
+
+    System.setProperty("curator-log-events", "true");
+    startSecureZK();
+    CuratorService curator = null;
+    LoginContext login = login(ZOOKEEPER_LOCALHOST,
+        ZOOKEEPER_CLIENT_CONTEXT,
+        keytab_zk);
+    try {
+      logLoginDetails(ZOOKEEPER, login);
+      RegistrySecurity.setZKSaslClientProperties(ZOOKEEPER,
+                                                ZOOKEEPER_CLIENT_CONTEXT);
+      curator = startCuratorServiceInstance("ZK", true);
+      LOG.info(curator.toString());
+
+      addToTeardown(curator);
+      curator.zkMkPath("/", CreateMode.PERSISTENT, false,
+          RegistrySecurity.WorldReadWriteACL);
+      curator.zkList("/");
+      curator.zkMkPath("/zookeeper", CreateMode.PERSISTENT, false,
+          RegistrySecurity.WorldReadWriteACL);
+    } finally {
+      logout(login);
+      ServiceOperations.stop(curator);
+    }
+  }
+
+  /**
+   * Start a curator service instance
+   * @param name name
+   * @param secure flag to indicate the cluster is secure
+   * @return an inited and started curator service
+   */
+  protected CuratorService startCuratorServiceInstance(String name,
+      boolean secure) {
+    Configuration clientConf = new Configuration();
+    clientConf.set(KEY_REGISTRY_ZK_ROOT, "/");
+    clientConf.setBoolean(KEY_REGISTRY_SECURE, secure);
+    describe(LOG, "Starting Curator service");
+    CuratorService curatorService = new CuratorService(name, secureZK);
+    curatorService.init(clientConf);
+    curatorService.start();
+    LOG.info("Curator Binding {}",
+        curatorService.bindingDiagnosticDetails());
+    return curatorService;
+  }
+
+  /**
+   * have the ZK user create the root dir.
+   * This logs out the ZK user after and stops its curator instance,
+   * to avoid contamination
+   * @throws Throwable
+   */
+  public void userZookeeperToCreateRoot() throws Throwable {
+
+    System.setProperty("curator-log-events", "true");
+    CuratorService curator = null;
+    LoginContext login = login(ZOOKEEPER_LOCALHOST,
+        ZOOKEEPER_CLIENT_CONTEXT,
+        keytab_zk);
+    try {
+      logLoginDetails(ZOOKEEPER, login);
+      RegistrySecurity.setZKSaslClientProperties(ZOOKEEPER,
+          ZOOKEEPER_CLIENT_CONTEXT);
+      curator = startCuratorServiceInstance("ZK", true);
+      LOG.info(curator.toString());
+
+      addToTeardown(curator);
+      curator.zkMkPath("/", CreateMode.PERSISTENT, false,
+          RegistrySecurity.WorldReadWriteACL);
+      ZKPathDumper pathDumper = curator.dumpPath(true);
+      LOG.info(pathDumper.toString());
+    } finally {
+      logout(login);
+      ServiceOperations.stop(curator);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/resources/log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/resources/log4j.properties
new file mode 100644
index 0000000..629e1d6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/resources/log4j.properties
@@ -0,0 +1,63 @@
+# 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.
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=INFO,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
+
+log4j.appender.subprocess=org.apache.log4j.ConsoleAppender
+log4j.appender.subprocess.layout=org.apache.log4j.PatternLayout
+log4j.appender.subprocess.layout.ConversionPattern=[%c{1}]: %m%n
+
+# packages under test
+log4j.logger.org.apache.hadoop.yarn.registry=DEBUG
+log4j.logger.org.apache.hadoop.service=DEBUG
+
+log4j.logger.org.apache.hadoop.security.UserGroupInformation=DEBUG
+
+
+#crank back on some noise
+log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
+log4j.logger.org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceScanner=WARN
+log4j.logger.org.apache.hadoop.hdfs.server.blockmanagement=WARN
+log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=WARN
+log4j.logger.org.apache.hadoop.hdfs=WARN
+
+
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor=WARN
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl=WARN
+log4j.logger.org.apache.zookeeper=INFO
+log4j.logger.org.apache.zookeeper.ClientCnxn=DEBUG
+
+log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.security=WARN
+log4j.logger.org.apache.hadoop.metrics2=ERROR
+log4j.logger.org.apache.hadoop.util.HostsFileReader=WARN
+log4j.logger.org.apache.hadoop.yarn.event.AsyncDispatcher=WARN
+log4j.logger.org.apache.hadoop.security.token.delegation=WARN
+log4j.logger.org.apache.hadoop.yarn.util.AbstractLivelinessMonitor=WARN
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.security=WARN
+log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.RMNMInfo=WARN
+
+# curator noise
+log4j.logger.org.apache.curator.framework.imps=WARN
+log4j.logger.org.apache.curator.framework.state.ConnectionStateManager=ERROR
+
+log4j.logger.org.apache.directory.api.ldap=ERROR
+log4j.logger.org.apache.directory.server=ERROR
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/index.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/index.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/index.apt.vm
index 9f07b19..a60e4a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/index.apt.vm
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/index.apt.vm
@@ -58,3 +58,5 @@ MapReduce NextGen aka YARN aka MRv2
   * {{{../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduce_Compatibility_Hadoop1_Hadoop2.html}Backward Compatibility between Apache Hadoop 1.x and 2.x for MapReduce}}
 
   * {{{../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/EncryptedShuffle.html}Encrypted Shuffle}}
+
+  * {{{./registry/index.html}Registry}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/index.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/index.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/index.md
new file mode 100644
index 0000000..a9ea24f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/index.md
@@ -0,0 +1,28 @@
+<!---
+   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.
+-->
+
+# YARN Service Registry
+
+The Service registry is a service which can be deployed in a Hadoop cluster
+to allow deployed applications to register themselves and the means of
+communicating with them. Client applications can then locate services
+and use the binding information to connect with the services's network-accessible
+endpoints, be they REST, IPC, Web UI, Zookeeper quorum+path or some other protocol.
+
+* [Architecture](yarn-registry.html)
+* [Using the YARN Service registry](using-the-yarn-service-registry.html)
+* [Security](registry-security.html)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/registry-security.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/registry-security.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/registry-security.md
new file mode 100644
index 0000000..7278534
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/registry-security.md
@@ -0,0 +1,120 @@
+<!---
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+# Registry Security
+
+This document describes how security is implemented in the service registry
+
+In a non-Kerberos-enabled Hadoop cluster, the Registry does not offer any
+security at all: the registry is world writeable.
+
+This document is therefore relevant only to secure clusters.
+
+## Security Model
+
+The security model of the registry is designed to meet the following goals
+a secur
+1. Deliver functional security on e ZK installation.
+1. Allow the RM to create per-user regions of the registration space
+1. Allow applications belonging to a user to write registry entries
+into their part of the space. These may be short-lived or long-lived
+YARN applications,  or they may be be static applications.
+1. Prevent other users from writing into another user's part of the registry.
+1. Allow system services to register to a `/services` section of the registry.
+1. Provide read access to clients of a registry.
+1. Permit future support of DNS
+1. Permit the future support of registering data private to a user.
+This allows a service to publish binding credentials (keys &c) for clients to use.
+1. Not require a ZK keytab on every user's home directory in a YARN cluster.
+This implies that kerberos credentials cannot be used by YARN applications.
+
+
+ZK security uses an ACL model, documented in
+[Zookeeper and SASL](https://cwiki.apache.org/confluence/display/ZOOKEEPER/Zookeeper+and+SASL)
+In which different authentication schemes may be used to restrict access
+to different znodes. This permits the registry to use a mixed
+Kerberos + Private password model.
+
+* The YARN-based registry (the `RMRegistryOperationsService`), uses kerberos
+as the authentication mechanism for YARN itself.
+* The registry configures the base of the registry to be writeable only by
+itself and other hadoop system accounts holding the relevant kerberos credentials.
+* The user specific parts of the tree are also configured to allow the same
+system accounts to write and manipulate that part of the tree.
+* User accounts are created with a `(username,password)` keypair granted
+write access to their part of the tree.
+* The secret part of the keypair is stored in the users' home directory
+on HDFS, using the Hadoop Credentials API.
+* Initially, the entire registry tree will be world readable.
+
+
+What are the limitations of such a scheme?
+
+1. It is critical that the user-specific registry keypair is kept a secret.
+This relies on filesystem security to keep the file readable only
+ by the (authenticated) user.
+1. As the [ZK Documentation says](http://zookeeper.apache.org/doc/r3.4.6/zookeeperProgrammers.html#sc_ZooKeeperAccessControl),
+*" Authentication is done by sending the username:password in clear text"
+1. While it is possible to change the password for an account,
+this involves a recursive walk down the registry tree, and will stop all
+running services from being able to authenticate for write access until they
+reload the key.
+1. A world-readable registry tree is exposing information about the cluster.
+There is some mitigation here in that access may be restricted by IP Address.
+1. There's also the need to propagate information from the registry down to
+the clients for setting up ACLs.
+
+
+
+## ACL Configuration propagation
+
+The registry manager cannot rely on clients consistently setting
+ZK permissions. At the very least, they cannot relay on client applications
+unintentionally wrong values for the accounts of the system services
+
+*Solution*: Initially, a registry permission is used here.
+
+### Automatic domain extension
+
+In a kerberos domain, it is possible for a kerberized client to determine the
+realm of a cluster at run time from the local
+user's kerberos credentials as used to talk to YARN or HDFS.
+
+This can be used to auto-generate account names with the correct realm for the
+system accounts hence aid having valid constants.
+
+This allows the registry to support a default configuration value for
+`hadoop.registry.system.accounts` of:
+
+      "sasl:yarn@, sasl:mapred@, sasl:hdfs@, sasl:hadoop@";
+
+#### In-registry publishing of core binding data
+
+Another strategy could be to have a `ServiceRecord` at the root
+of the registry that actually defines the registry —including listing
+those default binding values in the `data` field..
+
+### Auditing
+
+Something (perhaps the RM) could scan a user's portion of the registry and
+detect some ACL problems: IP/world access too lax, admin account settings wrong.
+It cannot view or fix the ACL permissions unless it has the `ADMIN` permission,
+though that situation can at least be detected. Given the RM must have `DELETE`
+permissions further up the stack, it would be in a position to delete the errant
+part of the tree —though this could be a destructive overreaction.
+
+## Further Reading
+
+* [Zookeeper and SASL](https://cwiki.apache.org/confluence/display/ZOOKEEPER/Zookeeper+and+SASL)
+* [Up and Running with Secure Zookeeper](https://github.com/ekoontz/zookeeper/wiki)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/using-the-yarn-service-registry.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/using-the-yarn-service-registry.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/using-the-yarn-service-registry.md
new file mode 100644
index 0000000..dcae372
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/using-the-yarn-service-registry.md
@@ -0,0 +1,150 @@
+<!---
+   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.
+-->
+
+# Using the YARN Service Registry
+
+The YARN service registry can be used in a numbe of ways :-
+
+1. To register dynamic YARN-deployed applications with entries that match the lifespan of the YARN application. Service Records can be set to be deleted on the completion of the YARN application, the application attempt, or an individual container.
+1. To look up static or dynamic applications and the mechanisms to communicate with them. Those mechanisms can incude: HTTP(S) URLs, Zookeeper paths, hostnames and ports and even paths in a Hadoop filesystem to configuration data.
+1. On a secure cluster, to verify that a service binding has been published by a specific user, or a system account. This can be done simply by looking at the path under which an entry has been placed.
+1. To register static applications. These will remain in the registry until deleted. They can be updated as required.
+
+A user of the registry may be both a publisher of entries —Service Records— and a consumer of other services located via their service records. Different parts of a distributed application may also use it for different purposes -central manager services can publish bindings for use by the worker services, which can then look up the bindings to communicate with that manager even if it was restarted on different nodes in the cluster.
+
+The registry cannot be used:-
+
+* To subscribe to service records or registry paths and listen for changes.
+* To directly share arbitrary data from a server for their clients. Such data must be published by some other means, a means which the registry entry can publish.
+* To share secrets between processes. The registry is world readable.
+
+
+## Registry Application Design Patterns
+
+
+### Short-lived YARN Application Masters registering their public service endpoints.
+
+1. A YARN application is deployed. In a secure cluster, it is given the kerberos token to write to the registry.
+2. When launched, it creates a service record at a known path
+3. This record MAY have application attempt persistence policy of and an ID of the application attempt
+
+		yarn:persistence = "application_attempt"
+		yarn:id = ${application_attemptId}
+	This means that the record will be deleted when the application attempt completes, even if a new attempt is created. Every Application attempt will have to re-register the endpoint —which may be needed to locate the service anyway.
+4. Alternatively, the record MAY have the persistence policy of "application":
+
+		yarn:persistence = "application_attempt"
+		yarn:id = application_attemptId
+	This means that the record will persist even between application attempts, albeit with out of date endpoint information.
+5. Client applications look up the service by way of the path.
+
+The choice of path is an application specific one. For services with a yarn application name guaranteed to be unique, we recommend a convention of:
+
+	/users/${username}/applications/${service-class}/${instance-name}
+
+Alternatively, the application Id can be used in the path:
+
+	/users/${username}/applications/${service-class}/${applicationId}
+
+The latter makes mapping a YARN application listing entry to a service record trivial.
+
+Client applications may locate the service
+
+* By enumerating all instances of a service class and selecting one by specific critera.
+* From a supplied service class and instance name
+* If listed by application ID, from the service class and application ID.
+
+After locating a service record, the client may enumerate the `external` bindings and locate the entry with the desired API.
+
+
+### YARN Containers registering their public service endpoints
+
+Here all containers in a YARN application are publishing service endpoints for public consumption.
+
+1. The deployed containers are passed the base path under which they should register themselves.
+2. Long-lived containers must be passed an `id:password` pair which gives them the right to update these entries without the kerberos credentials of the user. This allows the containers to update their entries even after the user tokens granting the AM write access to a registry path expire.
+3. The containers instantiate a registry operations instance with the id:pass pair.
+4. They then a register service record on a path consisting of:
+
+		${base-path} + "/" + RegistryPathUtils.encodeYarnID(containerId)
+
+	This record should have the container persistence policy an ID of the container
+
+		yarn:persistence = "container"
+		yarn:id = containerId
+
+	When the container is terminated, the entry will be automatically deleted.
+
+5. The exported service endpoints of this container-deployed service should be listed in the  `external` endpoint list of the service record.
+6. Clients may enumerate all containers exported by a YARN application by listing the entries under `${base-path}`.
+
+
+### Registering Static cluster services.
+
+Services which are generally fixed in a cluster, but which need to publish binding and configuration information may be published in the registry. Example: an Apache Oozie service.
+Services external to the cluster to which deployed applications may also be published. Example: An Amazon Dynamo instance.
+
+
+These services can be registered under paths which belong to the users running the service, such as `/users/oozie` or `/users/hbase`. Client applications would use this path. While this can authenticate the validity of the service record, it does rely on the client applications knowing the username a service is deployed on, or being configured with the full path.
+
+The alternative is for the services to be deployed under a static services path, under `/services`. For example, `/services/oozie` could contain the registration of the Oozie service. As the permissions for this path are restricted to pre-configured system accounts, the presence of a service registration does, on a secure cluster, confirm that it was registered by the cluster administration tools.
+
+1. The service is deployed by some management tool, or directly by the cluster operator.
+2. The deployed application can register itself under its own user name if given the binding information for the registry.
+3. If the application is to be registered under `/services` and it has been deployed by one of the system user accounts —it may register itself directly.
+4. If the application does not have the permissions to do so, the cluster administration tools must register the service instead.
+5. Client applications may locate a service by resolving its well known/configured path.
+5. If a service is stopped, the administration tools may delete the entry, or retain the entry but delete all it service endpoints. This is a proposed convention to indicate "the service is known but not currently reachable".
+6. When a service is restarted, its binding information may be updated, or its entire registry entry recreated.
+
+
+### YARN containers locating their Application Master
+
+Here YARN containers register with their AM to receive work, usually by some heartbeat mechanism where they report in regularly. If the AM is configured for containers to outlive the application attempt, when an AM fails the containers keep running. These containers will need to bind to any restarted AM. They may also wish to conclude that if an AM does not restart, that they should eventually time out and terminate themselves. Such a policy helps the application react to network partitions.
+
+1. The YARN AM publishes its service endpoints such as the FQDN and socket port neede for IPC communications, or an HTTP/HTTPS URL needed for a REST channel. These are published in the `internal` endpoint list, with the specific API the containers use
+2. The YARN containers are launched with the path to the service record (somehow) passed to them. Environment variables or command line parameters are two viable mechanisms. Shared secrets may also be passed this way, so allowing the containers to authenticate themselves with the AM. An alternative means of publishing secrets is saving information to the filesystem in a path specified as one of the endpoints.
+2. The YARN containers look up the service registry to identify the communications binding.
+3. If the registered service entry cannot be found, the container MAY do one of: exit. spin with some (jittered) retry period, polling for the entry, until the entry reappears. This implies that the AM has been found.
+4. If the service entry is found, the client should attempt to communicate with the AM on its channel. Shared authentication details may be used to validate the client with the server and vice versa.
+5. The client report in to the AM until the connections start failing to connect or authenticate, or when a long lived connection is broken and cannot be restarted.
+6. A this point the client may revert to step (3). Again, some backoff policy with some jitter helps stop a newly-restarted AM being overloaded. Containers may also with to have some timeout after which they conclude that the AM is not coming back and exit.
+6. We recommend that alongside the functional commands that an AM may issue to a client, a "terminate" command can be issued to a container. This allows the system to handle the specific situation of the YARN Node Manager terminating while spawned containers keep running.
+
+### YARN Applications and containers publishing their management and metrics bindings
+
+Management ports and bindings are simply another endpoint to publish. These should be published as *internal* endpoints, as they are not intended for public consumption. By convention, the name of the management protocol shoudl be used as the endpoint's declared API: `JMX`, `ganglia`, etc.
+
+
+### Client application enumerating services by endpoint APIs
+
+A client application wishes to locate all services implementing a specific API, such as `"org.apache.hbase"`
+
+1. The client starts from a path in the registry
+1. The client calls `registryOperations.list(path)` to list all nodes directly under that path, getting a relative list of child nodes.
+1. the client enumerates the child record statuses by calling `stat()` on each child.
+1. For all status entries, if the size of the entry is > the value of `ServiceRecordHeader.getLength()`, it MAY contain a service record.
+1. The contents can be retrieved using the `resolve()` operation. If successful, it does contain a service record —so the client can enumerate the `external` endpoints and locate the one with the desired API.
+1. The `children` field of each `RegistryPathStatus` status entry should be examined. If it is >= 0, the enumeration should be performed recursively on the path of that entry.
+1. The operation ultimately completes with a list of all entries.
+
+This algorithm describes a depth first search of the registry tree. Variations are of course possible, including breadth first search, or immediately halting the search as soon as a single entry point. There is also the option of parallel searches of different subtrees —which may reduce search time, albeit at the price of a higher client load on the registry infrastructure.
+
+A Utility class `RegistryUtils` provides static utility methods for common registry operations,
+in particular, `RegistryUtils.listServiceRecords(registryOperations, path)`
+performs the listing and collection of all immediate child record entries of
+a specified path.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md
new file mode 100644
index 0000000..a2a5009
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md
@@ -0,0 +1,889 @@
+<!---
+   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.
+-->
+
+## The YARN Service Registry
+
+# Introduction and concepts
+
+This document describes a YARN service registry built to address a problem:
+*how can clients talk to YARN-deployed services and the components which form
+such services?*
+
+Service registration and discovery is a long-standing problem in distributed
+computing, dating back to Xerox's Grapevine Service. This proposal is for a
+registry for locating distributed applications deployed by YARN, and
+determining the binding information needed to communicate with these
+applications.
+
+### Definitions
+**Service**: a potentially-distributed application deployed in —or reachable
+from— a Hadoop YARN cluster. Examples: Apache HBase, Apache hcatalog, Apache
+Storm. Services may be short-lived or long-lived.
+
+**Service Class:** the name of a type of service, used as a path in a registry
+and matching the DNS-compatible path naming scheme. Examples:
+`org-apache-hbase`, `org-apache-hcatalog`
+
+**Component**: a distributed element of a service. Examples: HBase master
+nodes, HBase region servers and HBase REST servers.
+
+**Service Instance:** A single instance of an application. Example, an HBase
+cluster `demo1`. A service instance is running if the instances the components
+which for the service are running. This does not imply "live" in the
+distributed computing sense, merely that the process are running.
+
+**Component Instance**: a single instance of a component within a service
+instance. Examples: an HBase master node on host `rack1server6` or a region
+server on host `rack3server40`.
+
+**Endpoint**: one means of binding with a service instance or a component
+instance. Examples: HBase's Apache Zookeeper binding, a Java JMX port on a
+region server, a Web UI on an HBase Master, and the REST API of an HBase REST
+component instance. Endpoints may be *internal* —for use within the service
+instance, or *external*: for use by clients of a service instance.
+
+**Service Record**: a record in the registry describing a service instance or a
+component instance, including listing its endpoints.
+
+**YARN Resource Manager, "RM":** the YARN component which allows client
+applications to submit work (including requests to deploy service instances) to
+a YARN cluster. The RM retains state on all running applications.
+
+**YARN Application**: An application deployed via YARN. Every application
+instance has a unique application ID.
+
+**YARN Application Master, "AM":** the application-specific component which is
+scheduled and deployed by the RM. It has the duty of maintaining the internal
+state of the application, including requesting and managing all other component
+instances of this application instance. The YARN RM will detect the failure of
+the AM, and respond by rescheduling it.
+
+**YARN Container:** An allocation of resources, including CPU and RAM, for a
+component instance. The AM has the responsibility of requesting the containers
+its components need, and building the commands to instantiate its component
+instances onto allocated containers. Every allocated container has a unique
+container ID.
+
+## The binding problem
+Hadoop YARN allows applications to run on the Hadoop cluster. Some of these are
+batch jobs or queries that can managed via Yarn’s existing API using its
+application ID. In addition YARN can deploy ong-lived services instances such a
+pool of Apache Tomcat web servers or an Apache HBase cluster. YARN will deploy
+them across the cluster depending on the individual each component requirements
+and server availability. These service instances need to be discovered by
+clients; traditionally their IP added is registered in DNS or in some
+configuration file —but that is not feasible in YARN-deployed applications when
+neither the hostname nor network ports can be known in advance.
+
+As a result there is no easy way for clients to interact with dynamically
+deployed applications.
+
+YARN supports a rudimentary registry which allows YARN Application Masters to
+register a web URL and an IPC address. but is not sufficient for our purposes
+since it It does not allow any other *endpoints* to be registered —such as REST
+URLs, or zookeeper path or the endpoints of the tasks that the Application
+Master executes. Further, information that can be registered is mapped to the
+YARN application instance —a unique instance ID that changes whenever a YARN
+application is started. This makes it impossible to resolve binding information
+via a static reference to a named service, or to even probe for the existence
+of a service instance which is not currently live.
+
+# Use Cases
+
+## Service Name Examples
+
+
+Core Hadoop services.
+
+These may be deployed statically, dynamically via an account with the
+permissions to write to the `/services` path, or even registrations of remote
+services accessible from within the Hadoop cluster
+
+        /services/hdfs
+        /services/yarn
+        /services/oozie
+
+Yarn-deployed services belonging to individual users.
+
+        /users/joe/org-apache-hbase/demo1
+        /users/joe/org-apache-hbase/demo1/components/regionserver1
+
+## Registration Use Cases
+
+1. A Hadoop core service that is not running under YARN example: HDFS) can be
+registered in for discovery. This could be done by the service or by management
+tools..
+
+2. A long-lived application deployed by YARN registers itself for discovery by
+clients. The registration data is intended to outlive the application master,
+and perhaps the lifespan of a single deployment of the service instance.
+
+3. Component instances of a service register themselves, publishing internal
+binding information, such as JMX ports.
+
+4. A YARN-deployed application can bind to dependent service instances both
+static and dynamic. Example: a Tomcat web pool binding to the dynamic HBase
+service instance "/users/joe/services/hbase/demo1".
+
+5. Component Instances use the registry to bind to an internal endpoint of
+their application master, to which they heartbeat regularly.
+
+## Unsupported Registration use cases:
+
+1. A short-lived Yarn application is registered automatically in the registry,
+including all its containers. and unregistered when the job terminates.
+Short-lived applications with many containers will place excessive load on a
+registry. All YARN applications will be given the option of registering, but it
+will not be automatic —and application authors must be advised against
+registering short-lived containers.
+
+## Lookup Use Cases
+
+1. A client application looks up a dynamically deployed service instance whose
+user, service class and instance name is known, e.g.
+`/users/joe/services/hbase/demo1`, and retrieves the information needed to
+connect to the service
+
+2. A client application looks up a statically deployed Hadoop service Example:
+`/services/hdfs`.
+
+3. An Application Master enumerates all registered component instances,
+discovers their listed JMX ports, and, initializes own web UI, offers links to these
+endpoints.
+
+4. A user connects to a private HBase service instance at
+`/users/joe/services/hbase/demo1`.
+
+5. A user connects to the cluster’s HBase service at `/services/hbase`.
+
+6. A user looks up the binding information to a remote Hadoop cluster's
+filesystem at `/net/cluster4/services/hdfs`. The registration information
+includes the `webhdfs://` URL for the remote filesystem.
+
+7. A user lists their HBase service instances:
+
+        ls /users/joe/services/hbase
+
+8. User finds all Hbase services in the cluster:
+
+        find -endpointField.api=org.apache.hbase
+
+9. Possibly in future: looking up a service via DNS.
+
+This registry proposal is intended to support these use cases by providing a
+means for applications to register their service endpoints, and for clients to
+locate them.
+
+# Key Requirements of a Service Registry
+
+Allow dynamic registration of service instances
+
+  * YARN deployed services instances must be able register their bindings and be
+  discovered by clients.
+
+  * Core Hadoop service instances must be able to register their service
+  endpoints.
+
+  * The binding must be upgradable if the service moves or in case if HA fails
+  over.
+
+  * A service instance must be able to publish a variety of endpoints for a
+  service: Web UI, RPC, REST, Zookeeper, others. Furthermore one must also be
+  able register certificates and other public security information may be
+  published as part of a binding.
+
+Registry service properties:
+
+  * The registry must be highly available.
+
+  * Scale: many services and many clients in a large cluster. This will limit
+  how much data a service can publish.
+
+  * Ubiquity: we need this in every YARN cluster, whether physical, virtual or
+  in-cloud.
+
+  * Must support hierarchical namespace and names. The name convention must
+  match that of DNS so that we have the option of accessing the namespace via
+  DNS protocol at a later phase of the project.
+
+  * Registry API Language/Protocols
+
+  * Cross-language: independent of any language; client language != service
+
+  * REST API for reading registry data
+
+Access Control:
+
+  * Read access for all
+
+  * Write is restricted so that squatting and impersonation can be avoided.
+
+Remote accessibility: supports remote access even on clusters which are
+  only reachable via Apache Knox, or hosted in cloud environments.
+
+## Non-Requirements
+
+* The registry is not intended for liveness detection, leader-election or
+  perform other "shared consensual state" actions for an application itself,
+  other than potentially sharing binding information between component
+  instances.
+
+* The registry is not intended to be a store for arbitrary application state,
+  or for publishing configuration data other than binding information to
+  endpoints offered by a service and its components. Such use would overload
+  the registry and rapidly reach limits of what Zookeeper permits.
+
+# Architecture
+
+We propose a base registry service that binds string-names to records
+describing service and component instances. We plan to use ZK as the base name
+service since it supports many of the properties, We pick a part of the ZK
+namespace to be the root of the service registry ( default: `yarnRegistry`).
+
+On top this base implementation we build our registry service API and the
+naming conventions that Yarn will use for its services. The registry will be
+accessed by the registry API, not directly via ZK - ZK is just an
+implementation choice (although unlikely to change in the future).
+
+1. Services are registered by binding a **_path_** to a value called a
+**_Service Record_**. Paths are hierarchical and use `/` as the root as well as
+the separator.
+
+2. Service records are registered as persistent znodes. This ensures that the
+record remains present during planned and unplanned outages of the service, on
+the assumption that client code is resilient to transient outages.
+
+3. Each service instance’s service record lists the endpoints for its various
+protocols exported by that service instance.
+
+4. For each protocol endpoint it must contain
+
+     1. The *protocol* name including: Web, REST, IPC, zookeeper. (type:string)
+
+     2. Its *address*: the specific details used to locate this endpoint
+
+     3. Its *addressType*. This is the format of the binding string. (URL, ZK path,
+        hostname:port pair). For the predefined protocols, we will define what
+        format the binding string MUST be. Example: `protocol==REST` means binding
+        type is `URL`, `protocol==IPC` binding uses the addresstype `host/port`.
+
+     4. The *api*. This is the API offered by the endpoint, and is application
+        specific. examples: `org.apache.hadoop.namenode`,
+        `org.apache.hadoop.webhdfs`
+
+5. Endpoints may be *external* —for use by programs other than the service
+itself, and *internal* —for connecting components within the service instance.
+They will be listed in different sections of the Service Record to distinguish
+them.
+
+6. Core services will be registered using the following convention:
+`/services/{servicename}` e.g. `/services/hdfs`.
+
+7. Yarn services SHOULD be registered using the following convention:
+
+        /users/{username}/{serviceclass}/{instancename}
+
+6. Component instances SHOULD be registered under
+
+        /users/{username}/{serviceclass}/{instancename}/components/{componentname}
+
+
+8. Each of the user's services which follows this convention must have unique
+service class names,
+
+9. Each component instance must have a name that is unique for that service
+instance. For a YARN-deployed application, this can be trivially
+derived from the container ID.
+
+The requirements for unique names ensures that the path to a service instance
+or component instance is guaranteed to be unique, and that all instances of a
+specific service class can be enumerated by listing all children of the service
+class path.
+
+
+# Registry Model
+
+Service entries MUST be persistent —it is the responsibility of YARN and other
+tools to determine when a service entry is to be deleted.
+
+## Path Elements
+
+All path elements MUST match that of a lower-case entry in a hostname path as
+defined in RFC1123; the regular expression is:
+
+    ([a-z0-9]|([a-z0-9][a-z0-9\-]*[a-z0-9]))
+
+This policy will ensure that were the registry hierarchy ever to exported by a
+DNS service, all service classes and names would be valid.
+
+A complication arises with user names, as platforms may allow user names with
+spaces, high unicode and other characters in them. Such paths must be converted
+to valid DNS hostname entries using the punycode convention used for
+internationalized DNS.
+
+## Service Record
+
+A Service Record has some basic information and possibly empty lists of
+internal and external endpoints.
+
+### Service Record:
+
+A Service Record contains some basic informations and two lists of endpoints:
+one list for users of a service, one list for internal use within the
+application.
+
+
+    <table>
+      <tr>
+        <td>Name</td>
+        <td>Description</td>
+      </tr>
+      <tr>
+        <td>description: String</td>
+        <td>Human-readable description.</td>
+      </tr>
+      <tr>
+        <td>external: List[Endpoint]</td>
+        <td>A list of service endpoints for external callers.</td>
+      </tr>
+      <tr>
+        <td>internal: List[Endpoint]</td>
+        <td>A list of service endpoints for internal use within the service instance.</td>
+      </tr>
+    </table>
+
+### YARN Persistence policies
+
+The YARN Resource Manager integration integrates cleanup of service records
+as an application, attempt or container is completed.
+
+This allows service to register entries which have a lifespan bound to one of
+these aspects of YARN applications' lifecycles. This is a feature which is only
+supported when the RM has enabled its support, and would not apply to
+any use of the registry without the RM's participation.
+
+The attributes, `yarn:id` and `yarn:persistence` specify which records
+*and any child entries* may be deleted as the associated YARN components complete.
+
+
+The `yarn:id` field defines the application, attempt or container ID to match;
+the `yarn:persistence` attribute defines the trigger for record cleanup, and
+implicitly the type of the contents of the `yarn:id` field.
+
+These attributes use the prefix "`yarn:`" to indicate that their reliance on
+the YARN layer of the Hadoop cluster to implement the policy. If the registry
+were to run standalone —which is entirely possible— all records would be
+implicitly persistent.
+
+<table>
+  <tr>
+    <td>Name</td>
+    <td>Description</td>
+    <td>contents of `yarn:id` field</td>
+  </tr>
+  <tr>
+    <td>permanent</td>
+    <td>The record persists until removed manually.</td>
+    <td>(unused)</td>
+  </tr>
+  <tr>
+    <td>application</td>
+    <td>Remove when the YARN application defined in the id field terminates.</td>
+    <td>application ID</td>
+  </tr>
+  <tr>
+    <td>application-attempt</td>
+    <td>Remove when the current YARN application attempt finishes.</td>
+    <td>application attempt ID</td>
+  </tr>
+  <tr>
+    <td>container</td>
+    <td>Remove when the YARN container in the ID field finishes</td>
+    <td>container ID</td>
+  </tr>
+</table>
+
+
+The policies which clean up when an application, application attempt or
+container terminates require the `yarn:id` field to match that of the
+application, attempt or container. If the wrong ID is set, the cleanup does not
+take place —and if set to a different application or container, will be cleaned
+up according the lifecycle of that application.
+
+### Endpoint:
+
+<table>
+  <tr>
+    <td>Name</td>
+    <td>Description</td>
+  </tr>
+  <tr>
+    <td>addresses: List[List[String]]</td>
+    <td>a list of address tuples whose format depends on the address type</td>
+  </tr>
+  <tr>
+    <td>addressType: String</td>
+    <td>format of the binding</td>
+  </tr>
+  <tr>
+    <td>protocol: String</td>
+    <td>Protocol. Examples:
+`http`, `https`, `hadoop-rpc`, `zookeeper`, `web`, `REST`, `SOAP`, ...</td>
+  </tr>
+  <tr>
+    <td>api: String</td>
+    <td>API implemented at the end of the binding</td>
+  </tr>
+</table>
+
+
+All string fields have a limit on size, to dissuade services from hiding
+complex JSON structures in the text description.
+
+### Field: Address Type
+
+The addressType field defines the string format of entries.
+
+Having separate types is that tools (such as a web viewer) can process binding
+strings without having to recognize the protocol.
+
+<table>
+  <tr>
+    <td>Format</td>
+    <td>binding format</td>
+  </tr>
+  <tr>
+    <td>`url`</td>
+    <td>`[URL]`</td>
+  </tr>
+  <tr>
+    <td>`hostname`</td>
+    <td>`[hostname]`</td>
+  </tr>
+  <tr>
+    <td>`inetaddress`</td>
+    <td>`[hostname, port]`</td>
+  </tr>
+  <tr>
+    <td>`path`</td>
+    <td>`[/path/to/something]`</td>
+  </tr>
+  <tr>
+    <td>`zookeeper`</td>
+    <td>`[quorum-entry, path]`</td>
+  </tr>
+</table>
+
+
+An actual zookeeper binding consists of a list of `hostname:port` bindings –the
+quorum— and the path within. In the proposed schema, every quorum entry will be
+listed as a triple of `[hostname, port, path]`. Client applications do not
+expect the path to de be different across the quorum. The first entry in the
+list of quorum hosts MUST define the path to be used by all clients. Later
+entries SHOULD list the same path, though clients MUST ignore these.
+
+New Address types may be defined; if not standard please prefix with the
+character sequence `"x-"`.
+
+#### **Field: API**
+
+APIs may be unique to a service class, or may be common across by service
+classes. They MUST be given unique names. These MAY be based on service
+packages but MAY be derived from other naming schemes:
+
+### Examples of Service Entries
+
+Here is an example of a service entry for a YARN-deployed tomcat application.
+
+After creation and registration of the application, the registry looks as follows:
+
+    /users
+      /devteam
+       /org-apache-tomcat
+         /test1
+           /components
+             /container-1408631738011-0001-01-000002
+             /container-1408631738011-0001-01-000001
+
+The `/users/devteam/org-apache-tomcat/tomcat-test` service record describes the
+overall application. It exports the URL to a load balancer.
+
+    {
+      "description" : "tomcat-based web application",
+      "registrationTime" : 1408638082444,
+      "external" : [ {
+        "api" : "www",
+        "addressType" : "uri",
+        "protocolType" : "REST",
+        "addresses" : [ [ "http://loadbalancer/" ] [ "http://loadbalancer2/" ] ]
+      } ],
+      "internal" : [ ]
+    }
+
+The service instance is built from two component instances, each described with
+their container ID converted into a DNS-compatible hostname. The entries are
+marked as ephemeral. If the entries were set within the container, then when
+that container is released or if the component fails, the entries will be
+automatically removed. Accordingly, it's persistence policy is declared to be
+"3", container. The `yarn:id` field identifies the container whose completion
+will trigger the deletion of this entry
+
+    /users/devteam/org-apache-tomcat/test1/components/container-1408631738011-0001-01-000001
+
+    {
+      "registrationTime" : 1408638082445,
+      "yarn:id" : "container_1408631738011_0001_01_000001",
+      "yarn:persistence" : "3",
+      "description" : null,
+      "external" : [ {
+        "api" : "www",
+        "addressType" : "uri",
+        "protocolType" : "REST",
+        "addresses" : [ [ "http://rack4server3:43572" ] ]
+      } ],
+      "internal" : [ {
+        "api" : "jmx",
+        "addressType" : "host/port",
+        "protocolType" : "JMX",
+        "addresses" : [ [ "rack4server3", "43573" ] ]
+      } ]
+    }
+
+The component instances list their endpoints: the public REST API as an
+external endpoint, the JMX addresses as internal.
+
+    /users/devteam/org-apache-tomcat/test1/components/container-1408631738011-0001-01-000002
+
+    {
+      "registrationTime" : 1408638082445,
+      "yarn:id" : "container_1408631738011_0001_01_000002",
+      "yarn:persistence" : "3",
+      "description" : null,
+      "external" : [ {
+        "api" : "www",
+        "addressType" : "uri",
+        "protocolType" : "REST",
+        "addresses" : [ [ "http://rack1server28:35881" ] ]
+      } ],
+      "internal" : [ {
+        "api" : "jmx",
+        "addressType" : "host/port",
+        "protocolType" : "JMX",
+        "addresses" : [ [ "rack1server28", "35882" ] ]
+      } ]
+    }
+
+This information could be used by the (hypothetical) load balancer to enumerate
+the components and build a list of component instances to dispatch requests to.
+Similarly, a management application could enumerate all available component
+instances and their JMX ports, then connect to each to collect performance
+metrics.
+
+# Registry API
+
+Here is the registry API as seen from a Java application. The API is a thin
+layer above the ZK operations, essentially building up paths, reading, writing
+and updating entries, and enumerating children. The REST API is implemented
+inside a server and use this same API to implement its REST API.
+
+The exceptions that are listed are only a subset of possible exception —the
+interface merely lists those that have special meaning.
+
+All write operations must assume that they are communicating with a registry
+service with the consistency view of a Zookeeper client; read-only clients must
+assume that their view may be somewhat out of date.
+
+All clients must recognize that the registry is a shared resource and that
+it may change during a sequence of actions.
+
+### Registry Operations
+
+    public interface RegistryOperations extends Service {
+
+      /**
+       * Create a path.
+       *
+       * It is not an error if the path exists already, be it empty or not.
+       *
+       * The createParents flag also requests creating the parents.
+       * As entries in the registry can hold data while still having
+       * child entries, it is not an error if any of the parent path
+       * elements have service records.
+       *
+       * @param path path to create
+       * @param createParents also create the parents.
+       * @throws PathNotFoundException parent path is not in the registry.
+       * @throws InvalidPathnameException path name is invalid.
+       * @throws IOException Any other IO Exception.
+       * @return true if the path was created, false if it existed.
+       */
+      boolean mknode(String path, boolean createParents)
+          throws PathNotFoundException,
+          InvalidPathnameException,
+          IOException;
+
+      /**
+       * Set a service record to an entry
+       * @param path path to service record
+       * @param record service record service record to create/update
+       * @param createFlags creation flags
+       * @throws PathNotFoundException the parent path does not exist
+       * @throws FileAlreadyExistsException path exists but create flags
+       * do not include "overwrite"
+       * @throws InvalidPathnameException path name is invalid.
+       * @throws IOException Any other IO Exception.
+       */
+      void bind(String path, ServiceRecord record, int createFlags)
+          throws PathNotFoundException,
+          FileAlreadyExistsException,
+          InvalidPathnameException,
+          IOException;
+
+      /**
+       * Resolve the record at a path
+       * @param path path to service record
+       * @return the record
+       * @throws PathNotFoundException path is not in the registry.
+       * @throws InvalidPathnameException the path is invalid.
+       * @throws IOException Any other IO Exception
+       */
+
+      ServiceRecord resolve(String path) throws PathNotFoundException,
+          InvalidPathnameException,
+          IOException;
+
+      /**
+       * Get the status of a path
+       * @param path path to query
+       * @return the status of the path
+       * @throws PathNotFoundException path is not in the registry.
+       * @throws InvalidPathnameException the path is invalid.
+       * @throws IOException Any other IO Exception
+       */
+      RegistryPathStatus stat(String path)
+          throws PathNotFoundException,
+          InvalidPathnameException,
+          IOException;
+
+      /**
+       * Probe for a path existing.
+       * This is equivalent to {@link #stat(String)} with
+       * any failure downgraded to a
+       * @param path path to query
+       * @return true if the path was found
+       * @throws IOException
+       */
+      boolean exists(String path) throws IOException;
+
+     /**
+       * List all entries under a registry path
+       * @param path path to query
+       * @return a possibly empty list of the full path names of
+       * child entries
+       * @throws PathNotFoundException
+       * @throws InvalidPathnameException
+       * @throws IOException
+       */
+       List<String> list(String path) throws
+          PathNotFoundException,
+          InvalidPathnameException,
+          IOException;
+
+      /**
+       * Delete a path.
+       *
+       * If the operation returns without an error then the entry has been
+       * deleted.
+       * @param path path delete recursively
+       * @param recursive recursive flag
+       * @throws PathNotFoundException path is not in the registry.
+       * @throws InvalidPathnameException the path is invalid.
+       * @throws PathIsNotEmptyDirectoryException path has child entries, but
+       * recursive is false.
+       * @throws IOException Any other IO Exception
+       *
+       */
+      void delete(String path, boolean recursive)
+          throws PathNotFoundException,
+          PathIsNotEmptyDirectoryException,
+          InvalidPathnameException,
+          IOException;
+
+      /**
+       * Add a new write access entry to be added to node permissions in all
+       * future write operations of a session connected to a secure registry.
+       *
+       * This does not grant the session any more rights: if it lacked any write
+       * access, it will still be unable to manipulate the registry.
+       *
+       * In an insecure cluster, this operation has no effect.
+       * @param id ID to use
+       * @param pass password
+       * @return true if the accessor was added: that is, the registry connection
+       * uses permissions to manage access
+       * @throws IOException on any failure to build the digest
+       */
+      boolean addWriteAccessor(String id, String pass) throws IOException;
+
+      /**
+       * Clear all write accessors.
+       *
+       * At this point all standard permissions/ACLs are retained,
+       * including any set on behalf of the user
+       * Only  accessors added via {@link #addWriteAccessor(String, String)}
+       * are removed.
+       */
+      public void clearWriteAccessors();
+    }
+
+
+### `RegistryPathStatus`
+
+The `RegistryPathStatus` class summarizes the contents of a node in the registry.
+
+    public class RegistryPathStatus {
+
+      /**
+       * Short path in the registry to this entry
+       */
+      public String path;
+
+      /**
+       * Timestamp
+       */
+      public long time;
+
+      /**
+       * Entry size in bytes, as returned by the storage infrastructure.
+       * In zookeeper, even "empty" nodes have a non-zero size.
+       */
+      public long size;
+
+      /**
+       * Number of child nodes
+       */
+      public int children;
+    }
+
+
+## Security
+
+The registry will allow a service instance can only be registered under the
+path where it has permissions. Yarn will create directories with appropriate
+permissions for users where Yarn deployed services can be registered by a user.
+of the user account of the service instance. The admin will also create
+directories (such as `/services`) with appropriate permissions (where core Hadoop
+services can register themselves.
+
+There will no attempt to restrict read access to registry information. The
+services will protect inappropriate access by clients by requiring
+authentication and authorization. There is a *scope* field in a service record
+, but this is just a marker to say "internal API only", rather than a direct
+security restriction. (this is why "internal" and "external" are proposed, not
+"public" and "private").
+
+Rationale: the endpoints being registered would be discoverable through port
+scanning anyway. Having everything world-readable allows the REST API to have a
+simpler access model —and is consistent with DNS.
+
+On a secure cluster, ZK token renewal may become an issue for long-lived
+services —if their token expires their session may expire. Renewal of such
+tokens is not part of the API implementation —we may need to add a means to
+update the tokens of an instance of the registry operations class.
+
+### Security Policy Summary
+
+In an a non-Kerberos Zookeeper Cluster, no security policy is implemented.
+
+The registry is designed to be secured *on a kerberos-managed cluster*.
+
+* The registry root grants full rights to "system accounts":
+`mapred`, `hdfs`, `yarn` : `"rwcda"`; all other accounts, and anonymous access
+is read-only.
+
+* The permissions are similarly restricted for `/users`, and `/services/`
+
+* installations may extend or change these system accounts.
+
+* When an application belonging to a user is scheduled, YARN
+SHALL create an entry for that user `/users/${username}`.
+
+* This node will have full access to the system; the user the access rights:
+`"crd"`. That is, they may create or delete child nodes, but not write to
+their home node, —or alter its permissions.
+
+* Applications wishing to write to the registry must use a SASL connection
+to authenticate via Zookeeper,
+
+* Applications creating nodes in the user path MUST include the site-specified
+system accounts in the ACL list, with full access.
+
+* Applications creating nodes in the user path MUST include an ACL Which
+
+* Applications creating nodes in the user path MUST declare their own
+user identity as a `sasl:user@REALM` entry.
+
+* Applications creating nodes the user path MAY add extra `digest:` ACL tokens
+so as to give their services the ability to manipulate portions of the
+registry *without needing kerberos credentials*.
+
+The digest-driven authentication avoid the problem of credential renewal in
+long-lived applications. An YARN application may be passed the token to
+connect with the ZK service when launched. It can then create or update an
+entry, including a secret digest ACL in the permissions of nodes it creates.
+As a result, even after the credentials expire, it retains *some* access.
+
+Note that for this to be successful, the client will need to fall back
+session to *not* use SASL, instead using authentication id:pass credentials.
+
+
+## Out of cluster and cross-cluster access
+
+1. A client should be able to access the registry of another cluster in order
+to access services of that cluster. Detail of this need to further fleshed out.
+
+2. Firewall services such as Apache Knox can examine the internal set of
+published services, and publish a subset of their endpoints. They MAY implement
+a future REST API.
+
+# Limits
+
+**Entry Size**
+
+Zookeeper has a default limit of 1MB/node. If all endpoints of a service or
+component are stored in JSON attached to that node, then there is a total limit
+of 1MB of all endpoint registration data.
+
+To prevent this becoming a problem, the client API should implement strict
+limits on the maximum length of fields, with low limits on the addressType,
+protocol, and api fields, something longer on the description and addresses
+elements —along with a limit on the number of elements in the addresses field.
+
+**Name size**
+
+To support DNS in future, there must be a limit of 63 bytes on all path
+elements. For non-ASCII User names, this restriction implies that a shorter
+path may be a limit.
+
+**Rate of Update**
+
+A rapid rate of entry change is considered antisocial in a ZK cluster.
+Implementations may throttle update operations.
+
+**Rate of Polling**
+
+Clients which poll the registry may be throttled.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/pom.xml b/hadoop-yarn-project/hadoop-yarn/pom.xml
index 13553d5..52049cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/pom.xml
@@ -100,5 +100,6 @@
     <module>hadoop-yarn-applications</module>
     <module>hadoop-yarn-site</module>
     <module>hadoop-yarn-client</module>
+    <module>hadoop-yarn-registry</module>
   </modules>
 </project>


[6/6] git commit: YARN-913 service registry: YARN-2652 add hadoop-yarn-registry package under hadoop-yarn

Posted by st...@apache.org.
YARN-913 service registry: YARN-2652 add hadoop-yarn-registry package under hadoop-yarn


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/088ae9c5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/088ae9c5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/088ae9c5

Branch: refs/heads/branch-2.6
Commit: 088ae9c5bf21aa5bee45f1fb304ff80ad008f7aa
Parents: 7ec65c8
Author: Steve Loughran <st...@apache.org>
Authored: Wed Oct 8 12:54:37 2014 -0700
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Oct 8 12:54:37 2014 -0700

----------------------------------------------------------------------
 .gitignore                                      |   2 +
 hadoop-project/pom.xml                          |   5 +
 hadoop-yarn-project/CHANGES.txt                 |   7 +
 .../dev-support/findbugs-exclude.xml            |  10 +
 .../src/main/resources/yarn-default.xml         | 126 +++
 .../hadoop-yarn/hadoop-yarn-registry/pom.xml    | 218 ++++
 .../apache/hadoop/registry/cli/RegistryCli.java | 445 +++++++++
 .../hadoop/registry/client/api/BindFlags.java   |  41 +
 .../registry/client/api/RegistryConstants.java  | 286 ++++++
 .../registry/client/api/RegistryOperations.java | 182 ++++
 .../client/api/RegistryOperationsFactory.java   | 131 +++
 .../registry/client/api/package-info.java       |  35 +
 .../registry/client/binding/JsonSerDeser.java   | 327 ++++++
 .../client/binding/RegistryPathUtils.java       | 218 ++++
 .../client/binding/RegistryTypeUtils.java       | 240 +++++
 .../registry/client/binding/RegistryUtils.java  | 362 +++++++
 .../registry/client/binding/package-info.java   |  22 +
 .../AuthenticationFailedException.java          |  39 +
 .../exceptions/InvalidPathnameException.java    |  40 +
 .../exceptions/InvalidRecordException.java      |  41 +
 .../NoChildrenForEphemeralsException.java       |  48 +
 .../exceptions/NoPathPermissionsException.java  |  45 +
 .../client/exceptions/NoRecordException.java    |  51 +
 .../client/exceptions/RegistryIOException.java  |  58 ++
 .../client/exceptions/package-info.java         |  33 +
 .../client/impl/RegistryOperationsClient.java   |  55 +
 .../registry/client/impl/package-info.java      |  26 +
 .../client/impl/zk/BindingInformation.java      |  41 +
 .../registry/client/impl/zk/CuratorService.java | 769 ++++++++++++++
 .../client/impl/zk/RegistryBindingSource.java   |  36 +
 .../impl/zk/RegistryInternalConstants.java      |  81 ++
 .../impl/zk/RegistryOperationsService.java      | 155 +++
 .../client/impl/zk/RegistrySecurity.java        | 996 +++++++++++++++++++
 .../registry/client/impl/zk/ZKPathDumper.java   | 133 +++
 .../client/impl/zk/ZookeeperConfigOptions.java  | 119 +++
 .../registry/client/impl/zk/package-info.java   |  39 +
 .../registry/client/types/AddressTypes.java     |  92 ++
 .../hadoop/registry/client/types/Endpoint.java  | 190 ++++
 .../registry/client/types/ProtocolTypes.java    | 104 ++
 .../client/types/RegistryPathStatus.java        | 123 +++
 .../registry/client/types/ServiceRecord.java    | 249 +++++
 .../client/types/ServiceRecordHeader.java       |  59 ++
 .../registry/client/types/package-info.java     |  41 +
 .../client/types/yarn/PersistencePolicies.java  |  50 +
 .../types/yarn/YarnRegistryAttributes.java      |  31 +
 .../RMRegistryOperationsService.java            | 246 +++++
 .../integration/SelectByYarnPersistence.java    |  60 ++
 .../server/integration/package-info.java        |  23 +
 .../hadoop/registry/server/package-info.java    |  27 +
 .../server/services/AddingCompositeService.java |  56 ++
 .../services/DeleteCompletionCallback.java      |  58 ++
 .../server/services/MicroZookeeperService.java  | 282 ++++++
 .../services/MicroZookeeperServiceKeys.java     |  69 ++
 .../server/services/RegistryAdminService.java   | 529 ++++++++++
 .../registry/server/services/package-info.java  |  40 +
 .../src/main/resources/.keep                    |   0
 .../src/main/tla/yarnregistry.tla               | 538 ++++++++++
 .../hadoop/registry/AbstractRegistryTest.java   | 123 +++
 .../hadoop/registry/AbstractZKRegistryTest.java | 113 +++
 .../hadoop/registry/RegistryTestHelper.java     | 401 ++++++++
 .../client/binding/TestMarshalling.java         | 121 +++
 .../binding/TestRegistryOperationUtils.java     |  47 +
 .../client/binding/TestRegistryPathUtils.java   | 178 ++++
 .../client/impl/CuratorEventCatcher.java        |  68 ++
 .../client/impl/TestCuratorService.java         | 249 +++++
 .../client/impl/TestMicroZookeeperService.java  |  60 ++
 .../integration/TestRegistryRMOperations.java   | 369 +++++++
 .../integration/TestYarnPolicySelector.java     |  65 ++
 .../operations/TestRegistryOperations.java      | 304 ++++++
 .../secure/AbstractSecureRegistryTest.java      | 356 +++++++
 .../registry/secure/KerberosConfiguration.java  |  81 ++
 .../secure/TestRegistrySecurityHelper.java      | 211 ++++
 .../registry/secure/TestSecureLogins.java       | 214 ++++
 .../secure/TestSecureRMRegistryOperations.java  | 350 +++++++
 .../registry/secure/TestSecureRegistry.java     | 157 +++
 .../src/test/resources/log4j.properties         |  63 ++
 .../hadoop-yarn-site/src/site/apt/index.apt.vm  |   2 +
 .../src/site/markdown/registry/index.md         |  28 +
 .../site/markdown/registry/registry-security.md | 120 +++
 .../registry/using-the-yarn-service-registry.md | 150 +++
 .../src/site/markdown/registry/yarn-registry.md | 889 +++++++++++++++++
 hadoop-yarn-project/hadoop-yarn/pom.xml         |   1 +
 82 files changed, 13049 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index db58f6a..6ece6ca 100644
--- a/.gitignore
+++ b/.gitignore
@@ -15,3 +15,5 @@ hadoop-hdfs-project/hadoop-hdfs/downloads
 hadoop-hdfs-project/hadoop-hdfs-httpfs/downloads
 hadoop-common-project/hadoop-common/src/test/resources/contract-test-options.xml
 hadoop-tools/hadoop-openstack/src/test/resources/contract-test-options.xml
+hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/tla/yarnregistry.toolbox
+yarnregistry.pdf

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index cfe7236..c26d366 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -222,6 +222,11 @@
 
       <dependency>
         <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-yarn-registry</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-yarn-server-nodemanager</artifactId>
         <version>${project.version}</version>
       </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1f26389..54ca114 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -85,6 +85,9 @@ Release 2.6.0 - UNRELEASED
     YARN-1051. Add a system for creating reservations of cluster capacity.
     (see breakdown below)
 
+    YARN-913. Add a way to register long-lived services in a YARN cluster.
+    (stevel)
+
   IMPROVEMENTS
 
     YARN-2242. Improve exception information on AM launch crashes. (Li Lu 
@@ -530,6 +533,10 @@ Release 2.6.0 - UNRELEASED
 
     YARN-2649. Fixed TestAMRMRPCNodeUpdates test failure. (Ming Ma via jianhe)
 
+  BREAKDOWN OF YARN-913 SUBTASKS AND RELATED JIRAS
+
+    YARN-2652 Add hadoop-yarn-registry package under hadoop-yarn. (stevel)
+
 Release 2.5.1 - 2014-09-05
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index 0e6207b..6e82af0 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -363,4 +363,14 @@
     <Field name="reservationsContinueLooking" />
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
+
+  <!--
+  This code is meant to deserialize this way...subclasses of will need to
+  instantiate their own JsonSerDeser instances if they want to deserialize.
+  -->
+  <Match>
+    <Class name="org.apache.hadoop.registry.client.binding.JsonSerDeser"/>
+    <Bug pattern="UI_INHERITANCE_UNSAFE_GETRESOURCE"/>
+  </Match>
+
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 4c11bd0..75efc27 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -1342,4 +1342,130 @@
     <name>yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled</name>
     <value>false</value>
   </property>
+
+  <!-- YARN registry -->
+
+  <property>
+    <description>
+      Is the registry enabled: does the RM start it up,
+      create the user and system paths, and purge
+      service records when containers, application attempts
+      and applications complete
+    </description>
+    <name>hadoop.registry.rm.enabled</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+    </description>
+    <name>hadoop.registry.zk.root</name>
+    <value>/registry</value>
+  </property>
+
+  <property>
+    <description>
+      Zookeeper session timeout in milliseconds
+    </description>
+    <name>hadoop.registry.zk.session.timeout.ms</name>
+    <value>60000</value>
+  </property>
+
+  <property>
+    <description>
+      Zookeeper session timeout in milliseconds
+    </description>
+    <name>hadoop.registry.zk.connection.timeout.ms</name>
+    <value>15000</value>
+  </property>
+
+  <property>
+    <description>
+      Zookeeper connection retry count before failing
+    </description>
+    <name>hadoop.registry.zk.retry.times</name>
+    <value>5</value>
+  </property>
+
+  <property>
+    <description>
+    </description>
+    <name>hadoop.registry.zk.retry.interval.ms</name>
+    <value>1000</value>
+  </property>
+
+  <property>
+    <description>
+      Zookeeper retry limit in milliseconds, during
+      exponential backoff: {@value}
+
+      This places a limit even
+      if the retry times and interval limit, combined
+      with the backoff policy, result in a long retry
+      period
+    </description>
+    <name>hadoop.registry.zk.retry.ceiling.ms</name>
+    <value>60000</value>
+  </property>
+
+  <property>
+    <description>
+      List of hostname:port pairs defining the
+      zookeeper quorum binding for the registry
+    </description>
+    <name>hadoop.registry.zk.quorum</name>
+    <value>localhost:2181</value>
+  </property>
+
+  <property>
+    <description>
+      Key to set if the registry is secure. Turning it on
+      changes the permissions policy from "open access"
+      to restrictions on kerberos with the option of
+      a user adding one or more auth key pairs down their
+      own tree.
+    </description>
+    <name>hadoop.registry.secure</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <description>
+      A comma separated list of Zookeeper ACL identifiers with
+      system access to the registry in a secure cluster.
+
+      These are given full access to all entries.
+
+      If there is an "@" at the end of a SASL entry it
+      instructs the registry client to append the default kerberos domain.
+    </description>
+    <name>hadoop.registry.system.acls</name>
+    <value>sasl:yarn@, sasl:mapred@, sasl:mapred@hdfs@</value>
+  </property>
+
+  <property>
+    <description>
+      The kerberos realm: used to set the realm of
+      system principals which do not declare their realm,
+      and any other accounts that need the value.
+
+      If empty, the default realm of the running process
+      is used.
+
+      If neither are known and the realm is needed, then the registry
+      service/client will fail.
+    </description>
+    <name>hadoop.registry.kerberos.realm</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+      Key to define the JAAS context. Used in secure
+      mode
+    </description>
+    <name>hadoop.registry.jaas.context</name>
+    <value>Client</value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml
new file mode 100644
index 0000000..ed80c42
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml
@@ -0,0 +1,218 @@
+<?xml version="1.0"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+                      http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>hadoop-yarn</artifactId>
+    <groupId>org.apache.hadoop</groupId>
+    <version>2.6.0-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>hadoop-yarn-registry</artifactId>
+  <version>2.6.0-SNAPSHOT</version>
+  <name>hadoop-yarn-registry</name>
+
+  <properties>
+    <!-- Needed for generating FindBugs warnings using parent pom -->
+    <yarn.basedir>${project.parent.basedir}</yarn.basedir>
+  </properties>
+
+  <dependencies>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+    </dependency>
+
+    <!-- needed for TimedOutTestsListener -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- Mini KDC is used for testing -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minikdc</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>commons-httpclient</groupId>
+      <artifactId>commons-httpclient</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-framework</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-test</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
+
+  <build>
+    <!--
+    Include all files in src/main/resources.  By default, do not apply property
+    substitution (filtering=false), but do apply property substitution to
+    yarn-version-info.properties (filtering=true).  This will substitute the
+    version information correctly, but prevent Maven from altering other files
+    like yarn-default.xml.
+    -->
+    <resources>
+      <resource>
+        <directory>${basedir}/src/main/resources</directory>
+        <excludes>
+          <exclude>yarn-version-info.properties</exclude>
+        </excludes>
+        <filtering>false</filtering>
+      </resource>
+      <resource>
+        <directory>${basedir}/src/main/resources</directory>
+        <includes>
+          <include>yarn-version-info.properties</include>
+        </includes>
+        <filtering>true</filtering>
+      </resource>
+    </resources>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+          <excludes>
+          </excludes>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-maven-plugins</artifactId>
+        <executions>
+          <execution>
+            <id>version-info</id>
+            <phase>generate-resources</phase>
+            <goals>
+              <goal>version-info</goal>
+            </goals>
+            <configuration>
+              <source>
+                <directory>${basedir}/src/main</directory>
+                <includes>
+                  <include>java/**/*.java</include>
+                  <!--
+                  <include>proto/**/*.proto</include>
+                    -->
+                </includes>
+              </source>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <phase>test-compile</phase>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+      <groupId>org.apache.maven.plugins</groupId>
+      <artifactId>maven-surefire-plugin</artifactId>
+      <configuration>
+        <reuseForks>false</reuseForks>
+        <forkedProcessTimeoutInSeconds>900</forkedProcessTimeoutInSeconds>
+        <argLine>-Xmx1024m -XX:+HeapDumpOnOutOfMemoryError</argLine>
+        <environmentVariables>
+          <!-- HADOOP_HOME required for tests on Windows to find winutils -->
+          <HADOOP_HOME>${hadoop.common.build.dir}</HADOOP_HOME>
+          <!-- configurable option to turn JAAS debugging on during test runs -->
+          <HADOOP_JAAS_DEBUG>true</HADOOP_JAAS_DEBUG>
+          <LD_LIBRARY_PATH>${env.LD_LIBRARY_PATH}:${project.build.directory}/native/target/usr/local/lib:${hadoop.common.build.dir}/native/target/usr/local/lib</LD_LIBRARY_PATH>
+          <MALLOC_ARENA_MAX>4</MALLOC_ARENA_MAX>
+        </environmentVariables>
+        <systemPropertyVariables>
+
+
+          <!-- TODO: all references in testcases should be updated to this default -->
+          <test.build.dir>${test.build.dir}</test.build.dir>
+          <hadoop.tmp.dir>${hadoop.tmp.dir}</hadoop.tmp.dir>
+          <test.build.data>${test.build.data}</test.build.data>
+          <test.build.webapps>${test.build.webapps}</test.build.webapps>
+          <test.cache.data>${test.cache.data}</test.cache.data>
+          <hadoop.log.dir>${hadoop.log.dir}</hadoop.log.dir>
+          <test.build.classes>${test.build.classes}</test.build.classes>
+
+          <java.net.preferIPv4Stack>true</java.net.preferIPv4Stack>
+          <java.security.krb5.conf>${basedir}/src/test/resources/krb5.conf</java.security.krb5.conf>
+          <java.security.egd>${java.security.egd}</java.security.egd>
+          <require.test.libhadoop>${require.test.libhadoop}</require.test.libhadoop>
+        </systemPropertyVariables>
+        <includes>
+          <include>**/Test*.java</include>
+        </includes>
+        <excludes>
+          <exclude>**/${test.exclude}.java</exclude>
+          <exclude>${test.exclude.pattern}</exclude>
+          <exclude>**/Test*$*.java</exclude>
+        </excludes>
+      </configuration>
+    </plugin>
+
+
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/cli/RegistryCli.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/cli/RegistryCli.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/cli/RegistryCli.java
new file mode 100644
index 0000000..863039e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/cli/RegistryCli.java
@@ -0,0 +1,445 @@
+/*
+ * 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.hadoop.registry.cli;
+
+import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.*;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.registry.client.api.BindFlags;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.api.RegistryOperationsFactory;
+import org.apache.hadoop.registry.client.exceptions.AuthenticationFailedException;
+import org.apache.hadoop.registry.client.exceptions.InvalidPathnameException;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.registry.client.exceptions.NoPathPermissionsException;
+import org.apache.hadoop.registry.client.exceptions.NoRecordException;
+import org.apache.hadoop.registry.client.types.Endpoint;
+import org.apache.hadoop.registry.client.types.ProtocolTypes;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RegistryCli extends Configured implements Tool {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RegistryCli.class);
+	protected final PrintStream sysout;
+	protected final PrintStream syserr;
+
+
+	private RegistryOperations registry;
+
+  static final String LS_USAGE = "ls pathName";
+  static final String RESOLVE_USAGE = "resolve pathName";
+  static final String BIND_USAGE =
+      "bind -inet  -api apiName -p portNumber -h hostName  pathName" + "\n"
+      + "bind -webui uriString -api apiName  pathName" + "\n"
+      + "bind -rest uriString -api apiName  pathName";
+  static final String MKNODE_USAGE = "mknode directoryName";
+  static final String RM_USAGE = "rm pathName";
+  static final String USAGE =
+      "\n" + LS_USAGE + "\n" + RESOLVE_USAGE + "\n" + BIND_USAGE + "\n" +
+      MKNODE_USAGE + "\n" + RM_USAGE;
+
+
+
+	public RegistryCli(PrintStream sysout, PrintStream syserr) {
+	    super(new YarnConfiguration());
+    this.sysout = sysout;
+    this.syserr = syserr;
+	}
+
+
+  @SuppressWarnings("UseOfSystemOutOrSystemErr")
+  public static void main(String[] args) throws Exception {
+    RegistryCli cli = new RegistryCli(System.out, System.err);
+    int res = ToolRunner.run(cli, args);
+    System.exit(res);
+  }
+
+  private int usageError(String err, String usage) {
+    syserr.println("Error: " + err);
+    syserr.println("Usage: " + usage);
+    return -1;
+  }
+
+  private boolean validatePath(String path) {
+    if (!path.startsWith("/")) {
+      syserr.println("Path must start with /; given path was: " + path);
+      return false;
+    }
+    return true;
+  }
+  @Override
+  public int run(String[] args) throws Exception {
+    Preconditions.checkArgument(getConf() != null, "null configuration");
+    registry = RegistryOperationsFactory.createInstance(
+        new YarnConfiguration(getConf()));
+    registry.start();
+    if (args.length > 0) {
+      if (args[0].equals("ls")) {
+        return ls(args);
+      } else if (args[0].equals("resolve")) {
+        return resolve(args);
+      } else if (args[0].equals("bind")) {
+        return bind(args);
+      } else if (args[0].equals("mknode")) {
+        return mknode(args);
+      } else if (args[0].equals("rm")) {
+        return rm(args);
+      }
+    }
+    return usageError("Invalid command: " + args[0], USAGE);
+  }
+
+  @SuppressWarnings("unchecked")
+	public int ls(String [] args) {
+
+		Options lsOption = new Options();
+		CommandLineParser parser = new GnuParser();
+		try {
+			CommandLine line = parser.parse(lsOption, args);
+
+			List<String> argsList = line.getArgList();
+			if (argsList.size() != 2) {
+				return usageError("ls requires exactly one path argument", LS_USAGE);
+		    }
+			if (!validatePath(argsList.get(1)))
+				return -1;
+
+			try {
+				List<String> children = registry.list(argsList.get(1));
+        for (String child : children) {
+          sysout.println(child);
+        }
+				return 0;
+
+      } catch (Exception e) {
+        syserr.println(analyzeException("ls", e, argsList));
+      }
+			return -1;
+		} catch (ParseException exp) {
+			return usageError("Invalid syntax " + exp, LS_USAGE);
+		}
+	}
+
+  @SuppressWarnings("unchecked")
+  public int resolve(String [] args) {
+		Options resolveOption = new Options();
+		CommandLineParser parser = new GnuParser();
+		try {
+			CommandLine line = parser.parse(resolveOption, args);
+
+			List<String> argsList = line.getArgList();
+			if (argsList.size() != 2) {
+				return usageError("resolve requires exactly one path argument", RESOLVE_USAGE);
+		    }
+			if (!validatePath(argsList.get(1)))
+				return -1;
+
+			try {
+				ServiceRecord record = registry.resolve(argsList.get(1));
+
+				for (Endpoint endpoint : record.external) {
+					if ((endpoint.protocolType.equals(ProtocolTypes.PROTOCOL_WEBUI))
+							|| (endpoint.protocolType.equals(ProtocolTypes.PROTOCOL_REST))) {
+						sysout.print(" Endpoint(ProtocolType="
+								+ endpoint.protocolType + ", Api="
+								+ endpoint.api + "); Uris are: ");
+					} else {
+						sysout.print(" Endpoint(ProtocolType="
+								+ endpoint.protocolType + ", Api="
+								+ endpoint.api + ");"
+								+ " Addresses(AddressType="
+								+ endpoint.addressType + ") are: ");
+
+					}
+					for (List<String> a : endpoint.addresses) {
+						sysout.print(a + " ");
+					}
+					sysout.println();
+				}
+				return 0;
+      } catch (Exception e) {
+        syserr.println(analyzeException("resolve", e, argsList));
+      }
+			return -1;
+		} catch (org.apache.commons.cli.ParseException exp) {
+			return usageError("Invalid syntax " + exp, RESOLVE_USAGE);
+		}
+
+	}
+
+	public int bind(String [] args) {
+		Option rest = OptionBuilder.withArgName("rest")
+				.hasArg()
+				.withDescription("rest Option")
+				.create("rest");
+		Option webui = OptionBuilder.withArgName("webui")
+				.hasArg()
+				.withDescription("webui Option")
+				.create("webui");
+		Option inet = OptionBuilder.withArgName("inet")
+				.withDescription("inet Option")
+				.create("inet");
+		Option port = OptionBuilder.withArgName("port")
+				.hasArg()
+				.withDescription("port to listen on [9999]")
+				.create("p");
+		Option host = OptionBuilder.withArgName("host")
+				.hasArg()
+				.withDescription("host name")
+				.create("h");
+		Option apiOpt = OptionBuilder.withArgName("api")
+				.hasArg()
+				.withDescription("api")
+				.create("api");
+		Options inetOption = new Options();
+		inetOption.addOption(inet);
+		inetOption.addOption(port);
+		inetOption.addOption(host);
+		inetOption.addOption(apiOpt);
+
+		Options webuiOpt = new Options();
+		webuiOpt.addOption(webui);
+		webuiOpt.addOption(apiOpt);
+
+		Options restOpt = new Options();
+		restOpt.addOption(rest);
+		restOpt.addOption(apiOpt);
+
+
+    CommandLineParser parser = new GnuParser();
+    ServiceRecord sr = new ServiceRecord();
+    CommandLine line = null;
+    if (args.length <= 1) {
+      return usageError("Invalid syntax ", BIND_USAGE);
+    }
+    if (args[1].equals("-inet")) {
+      int portNum;
+      String hostName;
+      String api;
+
+      try {
+        line = parser.parse(inetOption, args);
+      } catch (ParseException exp) {
+        return usageError("Invalid syntax " + exp.getMessage(), BIND_USAGE);
+      }
+      if (line.hasOption("inet") && line.hasOption("p") &&
+          line.hasOption("h") && line.hasOption("api")) {
+        portNum = Integer.parseInt(line.getOptionValue("p"));
+        hostName = line.getOptionValue("h");
+        api = line.getOptionValue("api");
+        sr.addExternalEndpoint(
+            inetAddrEndpoint(api, ProtocolTypes.PROTOCOL_HADOOP_IPC, hostName,
+                portNum));
+
+      } else {
+        return usageError("Missing options: must have host, port and api",
+            BIND_USAGE);
+      }
+
+    } else if (args[1].equals("-webui")) {
+      try {
+        line = parser.parse(webuiOpt, args);
+      } catch (ParseException exp) {
+        return usageError("Invalid syntax " + exp.getMessage(), BIND_USAGE);
+      }
+      if (line.hasOption("webui") && line.hasOption("api")) {
+        URI theUri = null;
+        try {
+          theUri = new URI(line.getOptionValue("webui"));
+        } catch (URISyntaxException e) {
+          return usageError("Invalid URI: " + e.getMessage(), BIND_USAGE);
+        }
+        sr.addExternalEndpoint(webEndpoint(line.getOptionValue("api"), theUri));
+
+      } else {
+        return usageError("Missing options: must have value for uri and api",
+            BIND_USAGE);
+      }
+    } else if (args[1].equals("-rest")) {
+      try {
+        line = parser.parse(restOpt, args);
+      } catch (ParseException exp) {
+        return usageError("Invalid syntax " + exp.getMessage(), BIND_USAGE);
+      }
+      if (line.hasOption("rest") && line.hasOption("api")) {
+        URI theUri = null;
+        try {
+          theUri = new URI(line.getOptionValue("rest"));
+        } catch (URISyntaxException e) {
+          return usageError("Invalid URI: " + e.getMessage(), BIND_USAGE);
+        }
+        sr.addExternalEndpoint(
+            restEndpoint(line.getOptionValue("api"), theUri));
+
+      } else {
+        return usageError("Missing options: must have value for uri and api",
+            BIND_USAGE);
+      }
+
+    } else {
+      return usageError("Invalid syntax", BIND_USAGE);
+    }
+    @SuppressWarnings("unchecked")
+		List<String> argsList = line.getArgList();
+		if (argsList.size() != 2) {
+			return usageError("bind requires exactly one path argument", BIND_USAGE);
+	    }
+		if (!validatePath(argsList.get(1)))
+			return -1;
+
+		try {
+			registry.bind(argsList.get(1), sr, BindFlags.OVERWRITE);
+			return 0;
+    } catch (Exception e) {
+      syserr.println(analyzeException("bind", e, argsList));
+    }
+
+    return -1;
+	}
+
+  @SuppressWarnings("unchecked")
+	public int mknode(String [] args) {
+		Options mknodeOption = new Options();
+		CommandLineParser parser = new GnuParser();
+		try {
+			CommandLine line = parser.parse(mknodeOption, args);
+
+			List<String> argsList = line.getArgList();
+			if (argsList.size() != 2) {
+				return usageError("mknode requires exactly one path argument", MKNODE_USAGE);
+		    }
+			if (!validatePath(argsList.get(1)))
+				return -1;
+
+			try {
+				registry.mknode(args[1], false);
+				return 0;
+			} catch (Exception e) {
+        syserr.println(analyzeException("mknode", e, argsList));
+			}
+			return -1;
+		} catch (ParseException exp) {
+			return usageError("Invalid syntax " + exp.toString(), MKNODE_USAGE);
+		}
+	}
+
+
+  @SuppressWarnings("unchecked")
+  public int rm(String[] args) {
+		Option recursive = OptionBuilder.withArgName("recursive")
+				.withDescription("delete recursively").create("r");
+
+		Options rmOption = new Options();
+		rmOption.addOption(recursive);
+
+		boolean recursiveOpt = false;
+
+		CommandLineParser parser = new GnuParser();
+		try {
+			CommandLine line = parser.parse(rmOption, args);
+
+			List<String> argsList = line.getArgList();
+			if (argsList.size() != 2) {
+				return usageError("RM requires exactly one path argument", RM_USAGE);
+		    }
+			if (!validatePath(argsList.get(1)))
+				return -1;
+
+			try {
+				if (line.hasOption("r")) {
+					recursiveOpt = true;
+				}
+
+				registry.delete(argsList.get(1), recursiveOpt);
+				return 0;
+      } catch (Exception e) {
+        syserr.println(analyzeException("rm", e, argsList));
+      }
+      return -1;
+		} catch (ParseException exp) {
+			return usageError("Invalid syntax " + exp.toString(), RM_USAGE);
+		}
+	}
+
+  /**
+   * Given an exception and a possibly empty argument list, generate
+   * a diagnostics string for use in error messages
+   * @param operation the operation that failed
+   * @param e exception
+   * @param argsList arguments list
+   * @return a string intended for the user
+   */
+  String analyzeException(String operation,
+      Exception e,
+      List<String> argsList) {
+
+    String pathArg = !argsList.isEmpty() ? argsList.get(1) : "(none)";
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Operation {} on path {} failed with exception {}",
+          operation, pathArg, e, e);
+    }
+    if (e instanceof InvalidPathnameException) {
+      return "InvalidPath :" + pathArg + ": " + e;
+    }
+    if (e instanceof PathNotFoundException) {
+      return "Path not found: " + pathArg;
+    }
+    if (e instanceof NoRecordException) {
+      return "No service record at path " + pathArg;
+    }
+    if (e instanceof AuthenticationFailedException) {
+      return "Failed to authenticate to registry : " + e;
+    }
+    if (e instanceof NoPathPermissionsException) {
+      return "No Permission to path: " + pathArg + ": " + e;
+    }
+    if (e instanceof AccessControlException) {
+      return "No Permission to path: " + pathArg + ": " + e;
+    }
+    if (e instanceof InvalidRecordException) {
+      return "Unable to read record at: " + pathArg + ": " + e;
+    }
+    if (e instanceof IOException) {
+      return "IO Exception when accessing path :" + pathArg + ": " + e;
+    }
+    // something else went very wrong here
+    return "Exception " + e;
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/BindFlags.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/BindFlags.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/BindFlags.java
new file mode 100644
index 0000000..5fd2aef
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/BindFlags.java
@@ -0,0 +1,41 @@
+/*
+ * 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.hadoop.registry.client.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Combinable Flags to use when creating a service entry.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface BindFlags {
+
+  /**
+   * Create the entry.. This is just "0" and can be "or"ed with anything
+   */
+  int CREATE = 0;
+
+  /**
+   * The entry should be created even if an existing entry is there.
+   */
+  int OVERWRITE = 1;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryConstants.java
new file mode 100644
index 0000000..a6fe216
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryConstants.java
@@ -0,0 +1,286 @@
+/*
+ * 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.hadoop.registry.client.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Constants for the registry, including configuration keys and default
+ * values.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface RegistryConstants {
+
+  /**
+   * prefix for registry configuration options: {@value}.
+   * Why <code>hadoop.</code> and not YARN? It can
+   * live outside YARN
+   */
+  String REGISTRY_PREFIX = "hadoop.registry.";
+
+  /**
+   * Prefix for zookeeper-specific options: {@value}
+   *  <p>
+   * For clients using other protocols, these options are not supported.
+   */
+  String ZK_PREFIX = REGISTRY_PREFIX + "zk.";
+
+  /**
+   * flag to indicate whether or not the registry should
+   * be enabled in the RM: {@value}
+   */
+  String KEY_REGISTRY_ENABLED = REGISTRY_PREFIX + "rm.enabled";
+
+  /**
+   * Defaut value for enabling the registry in the RM: {@value}
+   */
+  boolean DEFAULT_REGISTRY_ENABLED = false;
+
+  /**
+   * Key to set if the registry is secure: {@value}.
+   * Turning it on changes the permissions policy from "open access"
+   * to restrictions on kerberos with the option of
+   * a user adding one or more auth key pairs down their
+   * own tree.
+   */
+  String KEY_REGISTRY_SECURE = REGISTRY_PREFIX + "secure";
+
+  /**
+   * Default registry security policy: {@value}.
+   */
+  boolean DEFAULT_REGISTRY_SECURE = false;
+
+  /**
+   * Root path in the ZK tree for the registry: {@value}
+   */
+  String KEY_REGISTRY_ZK_ROOT = ZK_PREFIX + "root";
+
+  /**
+   * Default root of the yarn registry: {@value}
+   */
+  String DEFAULT_ZK_REGISTRY_ROOT = "/registry";
+
+  /**
+   * Registry client authentication policy.
+   *  <p>
+   * This is only used in secure clusters.
+   *  <p>
+   * If the Factory methods of {@link RegistryOperationsFactory}
+   * are used, this key does not need to be set: it is set
+   * up based on the factory method used.
+   */
+  String KEY_REGISTRY_CLIENT_AUTH =
+      REGISTRY_PREFIX + "client.auth";
+
+  /**
+   * Registry client uses Kerberos: authentication is automatic from
+   * logged in user
+   */
+  String REGISTRY_CLIENT_AUTH_KERBEROS = "kerberos";
+
+  /**
+   * Username/password is the authentication mechanism.
+   * If set then both {@link #KEY_REGISTRY_CLIENT_AUTHENTICATION_ID}
+   * and {@link #KEY_REGISTRY_CLIENT_AUTHENTICATION_PASSWORD} must be set.
+   */
+  String REGISTRY_CLIENT_AUTH_DIGEST = "digest";
+
+  /**
+   * No authentication; client is anonymous
+   */
+  String REGISTRY_CLIENT_AUTH_ANONYMOUS = "";
+
+  /**
+   * Registry client authentication ID
+   * <p>
+   * This is only used in secure clusters with
+   * {@link #KEY_REGISTRY_CLIENT_AUTH} set to
+   * {@link #REGISTRY_CLIENT_AUTH_DIGEST}
+   *
+   */
+  String KEY_REGISTRY_CLIENT_AUTHENTICATION_ID =
+      KEY_REGISTRY_CLIENT_AUTH + ".id";
+
+  /**
+   * Registry client authentication password.
+   * <p>
+   * This is only used in secure clusters with the client set to
+   * use digest (not SASL or anonymouse) authentication.
+   *  <p>
+   * Specifically, {@link #KEY_REGISTRY_CLIENT_AUTH} set to
+   * {@link #REGISTRY_CLIENT_AUTH_DIGEST}
+   *
+   */
+  String KEY_REGISTRY_CLIENT_AUTHENTICATION_PASSWORD =
+      KEY_REGISTRY_CLIENT_AUTH + ".password";
+
+  /**
+   * List of hostname:port pairs defining the
+   * zookeeper quorum binding for the registry {@value}
+   */
+  String KEY_REGISTRY_ZK_QUORUM = ZK_PREFIX + "quorum";
+
+  /**
+   * The default zookeeper quorum binding for the registry: {@value}
+   */
+  String DEFAULT_REGISTRY_ZK_QUORUM = "localhost:2181";
+
+  /**
+   * Zookeeper session timeout in milliseconds: {@value}
+   */
+  String KEY_REGISTRY_ZK_SESSION_TIMEOUT =
+      ZK_PREFIX + "session.timeout.ms";
+
+  /**
+  * The default ZK session timeout: {@value}.
+  */
+  int DEFAULT_ZK_SESSION_TIMEOUT = 60000;
+
+  /**
+   * Zookeeper connection timeout in milliseconds: {@value}.
+   */
+  String KEY_REGISTRY_ZK_CONNECTION_TIMEOUT =
+      ZK_PREFIX + "connection.timeout.ms";
+
+  /**
+   * The default ZK connection timeout: {@value}.
+   */
+  int DEFAULT_ZK_CONNECTION_TIMEOUT = 15000;
+
+  /**
+   * Zookeeper connection retry count before failing: {@value}.
+   */
+  String KEY_REGISTRY_ZK_RETRY_TIMES = ZK_PREFIX + "retry.times";
+
+  /**
+   * The default # of times to retry a ZK connection: {@value}.
+   */
+  int DEFAULT_ZK_RETRY_TIMES = 5;
+
+  /**
+   * Zookeeper connect interval in milliseconds: {@value}.
+   */
+  String KEY_REGISTRY_ZK_RETRY_INTERVAL =
+      ZK_PREFIX + "retry.interval.ms";
+
+  /**
+   * The default interval between connection retries: {@value}.
+   */
+  int DEFAULT_ZK_RETRY_INTERVAL = 1000;
+
+  /**
+   * Zookeeper retry limit in milliseconds, during
+   * exponential backoff: {@value}.
+   *
+   * This places a limit even
+   * if the retry times and interval limit, combined
+   * with the backoff policy, result in a long retry
+   * period
+   *
+   */
+  String KEY_REGISTRY_ZK_RETRY_CEILING =
+      ZK_PREFIX + "retry.ceiling.ms";
+
+  /**
+   * Default limit on retries: {@value}.
+   */
+  int DEFAULT_ZK_RETRY_CEILING = 60000;
+
+  /**
+   * A comma separated list of Zookeeper ACL identifiers with
+   * system access to the registry in a secure cluster: {@value}.
+   *
+   * These are given full access to all entries.
+   *
+   * If there is an "@" at the end of an entry it
+   * instructs the registry client to append the kerberos realm as
+   * derived from the login and {@link #KEY_REGISTRY_KERBEROS_REALM}.
+   */
+  String KEY_REGISTRY_SYSTEM_ACCOUNTS = REGISTRY_PREFIX + "system.accounts";
+
+  /**
+   * Default system accounts given global access to the registry: {@value}.
+   */
+  String DEFAULT_REGISTRY_SYSTEM_ACCOUNTS =
+      "sasl:yarn@, sasl:mapred@, sasl:hdfs@, sasl:hadoop@";
+
+  /**
+   * A comma separated list of Zookeeper ACL identifiers with
+   * system access to the registry in a secure cluster: {@value}.
+   *
+   * These are given full access to all entries.
+   *
+   * If there is an "@" at the end of an entry it
+   * instructs the registry client to append the default kerberos domain.
+   */
+  String KEY_REGISTRY_USER_ACCOUNTS = REGISTRY_PREFIX + "user.accounts";
+
+  /**
+   * Default system acls: {@value}.
+   */
+  String DEFAULT_REGISTRY_USER_ACCOUNTS = "";
+
+  /**
+   * The kerberos realm: {@value}.
+   *
+   * This is used to set the realm of
+   * system principals which do not declare their realm,
+   * and any other accounts that need the value.
+   *
+   * If empty, the default realm of the running process
+   * is used.
+   *
+   * If neither are known and the realm is needed, then the registry
+   * service/client will fail.
+   */
+  String KEY_REGISTRY_KERBEROS_REALM = REGISTRY_PREFIX + "kerberos.realm";
+
+  /**
+   * Key to define the JAAS context. Used in secure registries: {@value}.
+   */
+  String KEY_REGISTRY_CLIENT_JAAS_CONTEXT = REGISTRY_PREFIX + "jaas.context";
+
+  /**
+   * default client-side registry JAAS context: {@value}
+   */
+  String DEFAULT_REGISTRY_CLIENT_JAAS_CONTEXT = "Client";
+
+  /**
+   *  path to users off the root: {@value}.
+   */
+  String PATH_USERS = "/users/";
+
+  /**
+   *  path to system services off the root : {@value}.
+   */
+  String PATH_SYSTEM_SERVICES = "/services/";
+
+  /**
+   *  path to system services under a user's home path : {@value}.
+   */
+  String PATH_USER_SERVICES = "/services/";
+
+  /**
+   *  path under a service record to point to components of that service:
+   *  {@value}.
+   */
+  String SUBPATH_COMPONENTS = "/components/";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryOperations.java
new file mode 100644
index 0000000..c51bcf7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryOperations.java
@@ -0,0 +1,182 @@
+/*
+ * 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.hadoop.registry.client.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.registry.client.exceptions.InvalidPathnameException;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.registry.client.exceptions.NoRecordException;
+import org.apache.hadoop.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Registry Operations
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface RegistryOperations extends Service {
+
+  /**
+   * Create a path.
+   *
+   * It is not an error if the path exists already, be it empty or not.
+   *
+   * The createParents flag also requests creating the parents.
+   * As entries in the registry can hold data while still having
+   * child entries, it is not an error if any of the parent path
+   * elements have service records.
+   *
+   * @param path path to create
+   * @param createParents also create the parents.
+   * @throws PathNotFoundException parent path is not in the registry.
+   * @throws InvalidPathnameException path name is invalid.
+   * @throws IOException Any other IO Exception.
+   * @return true if the path was created, false if it existed.
+   */
+  boolean mknode(String path, boolean createParents)
+      throws PathNotFoundException,
+      InvalidPathnameException,
+      IOException;
+
+  /**
+   * Bind a path in the registry to a service record
+   * @param path path to service record
+   * @param record service record service record to create/update
+   * @param flags bind flags
+   * @throws PathNotFoundException the parent path does not exist
+   * @throws FileAlreadyExistsException path exists but create flags
+   * do not include "overwrite"
+   * @throws InvalidPathnameException path name is invalid.
+   * @throws IOException Any other IO Exception.
+   */
+  void bind(String path, ServiceRecord record, int flags)
+      throws PathNotFoundException,
+      FileAlreadyExistsException,
+      InvalidPathnameException,
+      IOException;
+
+  /**
+   * Resolve the record at a path
+   * @param path path to an entry containing a {@link ServiceRecord}
+   * @return the record
+   * @throws PathNotFoundException path is not in the registry.
+   * @throws NoRecordException if there is not a service record
+   * @throws InvalidRecordException if there was a service record but it could
+   * not be parsed.
+   * @throws IOException Any other IO Exception
+   */
+
+  ServiceRecord resolve(String path)
+      throws PathNotFoundException,
+      NoRecordException,
+      InvalidRecordException,
+      IOException;
+
+  /**
+   * Get the status of a path
+   * @param path path to query
+   * @return the status of the path
+   * @throws PathNotFoundException path is not in the registry.
+   * @throws InvalidPathnameException the path is invalid.
+   * @throws IOException Any other IO Exception
+   */
+  RegistryPathStatus stat(String path)
+      throws PathNotFoundException,
+      InvalidPathnameException,
+      IOException;
+
+  /**
+   * Probe for a path existing.
+   * This is equivalent to {@link #stat(String)} with
+   * any failure downgraded to a
+   * @param path path to query
+   * @return true if the path was found
+   * @throws IOException
+   */
+  boolean exists(String path) throws IOException;
+
+  /**
+   * List all entries under a registry path, returning the relative names
+   * of the entries.
+   * @param path path to query
+   * @return a possibly empty list of the short path names of
+   * child entries.
+   * @throws PathNotFoundException
+   * @throws InvalidPathnameException
+   * @throws IOException
+   */
+   List<String> list(String path) throws
+      PathNotFoundException,
+      InvalidPathnameException,
+      IOException;
+
+  /**
+   * Delete a path.
+   *
+   * If the operation returns without an error then the entry has been
+   * deleted.
+   * @param path path delete recursively
+   * @param recursive recursive flag
+   * @throws PathNotFoundException path is not in the registry.
+   * @throws InvalidPathnameException the path is invalid.
+   * @throws PathIsNotEmptyDirectoryException path has child entries, but
+   * recursive is false.
+   * @throws IOException Any other IO Exception
+   *
+   */
+  void delete(String path, boolean recursive)
+      throws PathNotFoundException,
+      PathIsNotEmptyDirectoryException,
+      InvalidPathnameException,
+      IOException;
+
+  /**
+   * Add a new write access entry to be added to node permissions in all
+   * future write operations of a session connected to a secure registry.
+   *
+   * This does not grant the session any more rights: if it lacked any write
+   * access, it will still be unable to manipulate the registry.
+   *
+   * In an insecure cluster, this operation has no effect.
+   * @param id ID to use
+   * @param pass password
+   * @return true if the accessor was added: that is, the registry connection
+   * uses permissions to manage access
+   * @throws IOException on any failure to build the digest
+   */
+  boolean addWriteAccessor(String id, String pass) throws IOException;
+
+  /**
+   * Clear all write accessors.
+   *
+   * At this point all standard permissions/ACLs are retained,
+   * including any set on behalf of the user
+   * Only  accessors added via {@link #addWriteAccessor(String, String)}
+   * are removed.
+   */
+  public void clearWriteAccessors();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryOperationsFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryOperationsFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryOperationsFactory.java
new file mode 100644
index 0000000..443654d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryOperationsFactory.java
@@ -0,0 +1,131 @@
+/*
+ * 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.hadoop.registry.client.api;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.ServiceStateException;
+import org.apache.hadoop.registry.client.impl.RegistryOperationsClient;
+
+import static org.apache.hadoop.registry.client.api.RegistryConstants.*;
+
+/**
+ * A factory for registry operation service instances.
+ * <p>
+ * <i>Each created instance will be returned initialized.</i>
+ * <p>
+ * That is, the service will have had <code>Service.init(conf)</code> applied
+ * to it —possibly after the configuration has been modified to
+ * support the specific binding/security mechanism used
+ */
+public final class RegistryOperationsFactory {
+
+  private RegistryOperationsFactory() {
+  }
+
+  /**
+   * Create and initialize a registry operations instance.
+   * Access writes will be determined from the configuration
+   * @param conf configuration
+   * @return a registry operations instance
+   * @throws ServiceStateException on any failure to initialize
+   */
+  public static RegistryOperations createInstance(Configuration conf) {
+    return createInstance("RegistryOperations", conf);
+  }
+
+  /**
+   * Create and initialize a registry operations instance.
+   * Access rights will be determined from the configuration
+   * @param name name of the instance
+   * @param conf configuration
+   * @return a registry operations instance
+   * @throws ServiceStateException on any failure to initialize
+   */
+  public static RegistryOperations createInstance(String name, Configuration conf) {
+    Preconditions.checkArgument(conf != null, "Null configuration");
+    RegistryOperationsClient operations =
+        new RegistryOperationsClient(name);
+    operations.init(conf);
+    return operations;
+  }
+
+  /**
+   * Create and initialize an anonymous read/write registry operations instance.
+   * In a secure cluster, this instance will only have read access to the
+   * registry.
+   * @param conf configuration
+   * @return an anonymous registry operations instance
+   *
+   * @throws ServiceStateException on any failure to initialize
+   */
+  public static RegistryOperations createAnonymousInstance(Configuration conf) {
+    Preconditions.checkArgument(conf != null, "Null configuration");
+    conf.set(KEY_REGISTRY_CLIENT_AUTH, REGISTRY_CLIENT_AUTH_ANONYMOUS);
+    return createInstance("AnonymousRegistryOperations", conf);
+  }
+
+  /**
+   * Create and initialize an secure, Kerberos-authenticated instance.
+   *
+   * The user identity will be inferred from the current user
+   *
+   * The authentication of this instance will expire when any kerberos
+   * tokens needed to authenticate with the registry infrastructure expire.
+   * @param conf configuration
+   * @param jaasContext the JAAS context of the account.
+   * @return a registry operations instance
+   * @throws ServiceStateException on any failure to initialize
+   */
+  public static RegistryOperations createKerberosInstance(Configuration conf,
+      String jaasContext) {
+    Preconditions.checkArgument(conf != null, "Null configuration");
+    conf.set(KEY_REGISTRY_CLIENT_AUTH, REGISTRY_CLIENT_AUTH_KERBEROS);
+    conf.set(KEY_REGISTRY_CLIENT_JAAS_CONTEXT, jaasContext);
+    return createInstance("KerberosRegistryOperations", conf);
+  }
+
+  /**
+   * Create and initialize an operations instance authenticated with write
+   * access via an <code>id:password</code> pair.
+   *
+   * The instance will have the read access
+   * across the registry, but write access only to that part of the registry
+   * to which it has been give the relevant permissions.
+   * @param conf configuration
+   * @param id user ID
+   * @param password password
+   * @return a registry operations instance
+   * @throws ServiceStateException on any failure to initialize
+   * @throws IllegalArgumentException if an argument is invalid
+   */
+  public static RegistryOperations createAuthenticatedInstance(Configuration conf,
+      String id,
+      String password) {
+    Preconditions.checkArgument(!StringUtils.isEmpty(id), "empty Id");
+    Preconditions.checkArgument(!StringUtils.isEmpty(password), "empty Password");
+    Preconditions.checkArgument(conf != null, "Null configuration");
+    conf.set(KEY_REGISTRY_CLIENT_AUTH, REGISTRY_CLIENT_AUTH_DIGEST);
+    conf.set(KEY_REGISTRY_CLIENT_AUTHENTICATION_ID, id);
+    conf.set(KEY_REGISTRY_CLIENT_AUTHENTICATION_PASSWORD, password);
+    return createInstance("DigestRegistryOperations", conf);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/package-info.java
new file mode 100644
index 0000000..f5f844e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/package-info.java
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/**
+ * YARN Registry Client API.
+ *
+ * This package contains the core API for the YARN registry.
+ *
+ * <ol>
+ *   <li> Data types can be found in
+ * {@link org.apache.hadoop.registry.client.types}</li>
+ *   <li> Exceptions are listed in
+ * {@link org.apache.hadoop.registry.client.exceptions}</li>
+ *   <li> Classes to assist use of the registry are in
+ * {@link org.apache.hadoop.registry.client.binding}</li>
+ * </ol>
+ *
+ *
+ */
+package org.apache.hadoop.registry.client.api;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/JsonSerDeser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/JsonSerDeser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/JsonSerDeser.java
new file mode 100644
index 0000000..e086e36
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/JsonSerDeser.java
@@ -0,0 +1,327 @@
+/*
+ * 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.hadoop.registry.client.binding;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.registry.client.exceptions.NoRecordException;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.JsonParseException;
+import org.codehaus.jackson.JsonProcessingException;
+import org.codehaus.jackson.map.DeserializationConfig;
+import org.codehaus.jackson.map.JsonMappingException;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.SerializationConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Support for marshalling objects to and from JSON.
+ *  <p>
+ * It constructs an object mapper as an instance field.
+ * and synchronizes access to those methods
+ * which use the mapper
+ * @param <T> Type to marshal.
+ */
+@InterfaceAudience.Private()
+@InterfaceStability.Evolving
+public class JsonSerDeser<T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JsonSerDeser.class);
+  private static final String UTF_8 = "UTF-8";
+  public static final String E_NO_SERVICE_RECORD = "No service record at path";
+
+  private final Class<T> classType;
+  private final ObjectMapper mapper;
+  private final byte[] header;
+
+  /**
+   * Create an instance bound to a specific type
+   * @param classType class to marshall
+   * @param header byte array to use as header
+   */
+  public JsonSerDeser(Class<T> classType, byte[] header) {
+    Preconditions.checkArgument(classType != null, "null classType");
+    Preconditions.checkArgument(header != null, "null header");
+    this.classType = classType;
+    this.mapper = new ObjectMapper();
+    mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES,
+        false);
+    // make an immutable copy to keep findbugs happy.
+    byte[] h = new byte[header.length];
+    System.arraycopy(header, 0, h, 0, header.length);
+    this.header = h;
+  }
+
+  public String getName() {
+    return classType.getSimpleName();
+  }
+
+  /**
+   * Convert from JSON
+   *
+   * @param json input
+   * @return the parsed JSON
+   * @throws IOException IO
+   * @throws JsonMappingException failure to map from the JSON to this class
+   */
+  @SuppressWarnings("unchecked")
+  public synchronized T fromJson(String json)
+      throws IOException, JsonParseException, JsonMappingException {
+    try {
+      return mapper.readValue(json, classType);
+    } catch (IOException e) {
+      LOG.error("Exception while parsing json : " + e + "\n" + json, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Convert from a JSON file
+   * @param jsonFile input file
+   * @return the parsed JSON
+   * @throws IOException IO problems
+   * @throws JsonMappingException failure to map from the JSON to this class
+   */
+  @SuppressWarnings("unchecked")
+  public synchronized T fromFile(File jsonFile)
+      throws IOException, JsonParseException, JsonMappingException {
+    try {
+      return mapper.readValue(jsonFile, classType);
+    } catch (IOException e) {
+      LOG.error("Exception while parsing json file {}: {}", jsonFile, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Convert from a JSON file
+   * @param resource input file
+   * @return the parsed JSON
+   * @throws IOException IO problems
+   * @throws JsonMappingException failure to map from the JSON to this class
+   */
+  @SuppressWarnings({"IOResourceOpenedButNotSafelyClosed"})
+  public synchronized T fromResource(String resource)
+      throws IOException, JsonParseException, JsonMappingException {
+    InputStream resStream = null;
+    try {
+      resStream = this.getClass().getResourceAsStream(resource);
+      if (resStream == null) {
+        throw new FileNotFoundException(resource);
+      }
+      return mapper.readValue(resStream, classType);
+    } catch (IOException e) {
+      LOG.error("Exception while parsing json resource {}: {}", resource, e);
+      throw e;
+    } finally {
+      IOUtils.closeStream(resStream);
+    }
+  }
+
+  /**
+   * clone by converting to JSON and back again.
+   * This is much less efficient than any Java clone process.
+   * @param instance instance to duplicate
+   * @return a new instance
+   * @throws IOException problems.
+   */
+  public T fromInstance(T instance) throws IOException {
+    return fromJson(toJson(instance));
+  }
+
+  /**
+   * Load from a Hadoop filesystem
+   * @param fs filesystem
+   * @param path path
+   * @return a loaded CD
+   * @throws IOException IO problems
+   * @throws EOFException if not enough bytes were read in
+   * @throws JsonParseException parse problems
+   * @throws JsonMappingException O/J mapping problems
+   */
+  public T load(FileSystem fs, Path path)
+      throws IOException, JsonParseException, JsonMappingException {
+    FileStatus status = fs.getFileStatus(path);
+    long len = status.getLen();
+    byte[] b = new byte[(int) len];
+    FSDataInputStream dataInputStream = fs.open(path);
+    int count = dataInputStream.read(b);
+    if (count != len) {
+      throw new EOFException(path.toString() + ": read finished prematurely");
+    }
+    return fromBytes(path.toString(), b, 0);
+  }
+
+  /**
+   * Save a cluster description to a hadoop filesystem
+   * @param fs filesystem
+   * @param path path
+   * @param overwrite should any existing file be overwritten
+   * @throws IOException IO exception
+   */
+  public void save(FileSystem fs, Path path, T instance,
+      boolean overwrite) throws
+      IOException {
+    FSDataOutputStream dataOutputStream = fs.create(path, overwrite);
+    writeJsonAsBytes(instance, dataOutputStream);
+  }
+
+  /**
+   * Write the json as bytes -then close the file
+   * @param dataOutputStream an outout stream that will always be closed
+   * @throws IOException on any failure
+   */
+  private void writeJsonAsBytes(T instance,
+      DataOutputStream dataOutputStream) throws
+      IOException {
+    try {
+      byte[] b = toBytes(instance);
+      dataOutputStream.write(b);
+    } finally {
+      dataOutputStream.close();
+    }
+  }
+
+  /**
+   * Convert JSON To bytes
+   * @param instance instance to convert
+   * @return a byte array
+   * @throws IOException
+   */
+  public byte[] toBytes(T instance) throws IOException {
+    String json = toJson(instance);
+    return json.getBytes(UTF_8);
+  }
+
+  /**
+   * Convert JSON To bytes, inserting the header
+   * @param instance instance to convert
+   * @return a byte array
+   * @throws IOException
+   */
+  public byte[] toByteswithHeader(T instance) throws IOException {
+    byte[] body = toBytes(instance);
+
+    ByteBuffer buffer = ByteBuffer.allocate(body.length + header.length);
+    buffer.put(header);
+    buffer.put(body);
+    return buffer.array();
+  }
+
+  /**
+   * Deserialize from a byte array
+   * @param path path the data came from
+   * @param bytes byte array
+   * @return offset in the array to read from
+   * @throws IOException all problems
+   * @throws EOFException not enough data
+   * @throws InvalidRecordException if the parsing failed -the record is invalid
+   */
+  public T fromBytes(String path, byte[] bytes, int offset) throws IOException,
+      InvalidRecordException {
+    int data = bytes.length - offset;
+    if (data <= 0) {
+      throw new EOFException("No data at " + path);
+    }
+    String json = new String(bytes, offset, data, UTF_8);
+    try {
+      return fromJson(json);
+    } catch (JsonProcessingException e) {
+      throw new InvalidRecordException(path, e.toString(), e);
+    }
+  }
+
+  /**
+   * Read from a byte array to a type, checking the header first
+   * @param path source of data
+   * @param buffer buffer
+   * @return the parsed structure
+   * Null if the record was too short or the header did not match
+   * @throws IOException on a failure
+   * @throws NoRecordException if header checks implied there was no record
+   * @throws InvalidRecordException if record parsing failed
+   */
+  @SuppressWarnings("unchecked")
+  public T fromBytesWithHeader(String path, byte[] buffer) throws IOException {
+    int hlen = header.length;
+    int blen = buffer.length;
+    if (hlen > 0) {
+      if (blen < hlen) {
+        throw new NoRecordException(path, E_NO_SERVICE_RECORD);
+      }
+      byte[] magic = Arrays.copyOfRange(buffer, 0, hlen);
+      if (!Arrays.equals(header, magic)) {
+        LOG.debug("start of entry does not match service record header at {}",
+            path);
+        throw new NoRecordException(path, E_NO_SERVICE_RECORD);
+      }
+    }
+    return fromBytes(path, buffer, hlen);
+  }
+
+  /**
+   * Check if a buffer has a header which matches this record type
+   * @param buffer buffer
+   * @return true if there is a match
+   * @throws IOException
+   */
+  public boolean headerMatches(byte[] buffer) throws IOException {
+    int hlen = header.length;
+    int blen = buffer.length;
+    boolean matches = false;
+    if (blen > hlen) {
+      byte[] magic = Arrays.copyOfRange(buffer, 0, hlen);
+      matches = Arrays.equals(header, magic);
+    }
+    return matches;
+  }
+
+  /**
+   * Convert an object to a JSON string
+   * @param instance instance to convert
+   * @return a JSON string description
+   * @throws JsonParseException parse problems
+   * @throws JsonMappingException O/J mapping problems
+   */
+  public synchronized String toJson(T instance) throws IOException,
+      JsonGenerationException,
+      JsonMappingException {
+    mapper.configure(SerializationConfig.Feature.INDENT_OUTPUT, true);
+    return mapper.writeValueAsString(instance);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
new file mode 100644
index 0000000..5d8ea3f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
@@ -0,0 +1,218 @@
+/*
+ * 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.hadoop.registry.client.binding;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.registry.client.exceptions.InvalidPathnameException;
+import org.apache.hadoop.registry.client.impl.zk.RegistryInternalConstants;
+import org.apache.zookeeper.common.PathUtils;
+
+import java.net.IDN;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Pattern;
+
+/**
+ * Basic operations on paths: manipulating them and creating and validating
+ * path elements.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RegistryPathUtils {
+
+  /**
+   * Compiled down pattern to validate single entries in the path
+   */
+  private static final Pattern PATH_ENTRY_VALIDATION_PATTERN =
+      Pattern.compile(RegistryInternalConstants.VALID_PATH_ENTRY_PATTERN);
+
+  /**
+   * Validate ZK path with the path itself included in
+   * the exception text
+   * @param path path to validate
+   * @return the path parameter
+   * @throws InvalidPathnameException if the pathname is invalid.
+   */
+  public static String validateZKPath(String path) throws
+      InvalidPathnameException {
+    try {
+      PathUtils.validatePath(path);
+
+    } catch (IllegalArgumentException e) {
+      throw new InvalidPathnameException(path,
+          "Invalid Path \"" + path + "\" : " + e, e);
+    }
+    return path;
+  }
+
+  /**
+   * Validate ZK path as valid for a DNS hostname.
+   * @param path path to validate
+   * @return the path parameter
+   * @throws InvalidPathnameException if the pathname is invalid.
+   */
+  public static String validateElementsAsDNS(String path) throws
+      InvalidPathnameException {
+    List<String> splitpath = split(path);
+    for (String fragment : splitpath) {
+      if (!PATH_ENTRY_VALIDATION_PATTERN.matcher(fragment).matches()) {
+        throw new InvalidPathnameException(path,
+            "Invalid Path element \"" + fragment + "\"");
+      }
+    }
+    return path;
+  }
+
+  /**
+   * Create a full path from the registry root and the supplied subdir
+   * @param path path of operation
+   * @return an absolute path
+   * @throws InvalidPathnameException if the path is invalid
+   */
+  public static String createFullPath(String base, String path) throws
+      InvalidPathnameException {
+    Preconditions.checkArgument(path != null, "null path");
+    Preconditions.checkArgument(base != null, "null path");
+    return validateZKPath(join(base, path));
+  }
+
+  /**
+   * Join two paths, guaranteeing that there will not be exactly
+   * one separator between the two, and exactly one at the front
+   * of the path. There will be no trailing "/" except for the special
+   * case that this is the root path
+   * @param base base path
+   * @param path second path to add
+   * @return a combined path.
+   */
+  public static String join(String base, String path) {
+    Preconditions.checkArgument(path != null, "null path");
+    Preconditions.checkArgument(base != null, "null path");
+    StringBuilder fullpath = new StringBuilder();
+
+    if (!base.startsWith("/")) {
+      fullpath.append('/');
+    }
+    fullpath.append(base);
+
+    // guarantee a trailing /
+    if (!fullpath.toString().endsWith("/")) {
+      fullpath.append("/");
+    }
+    // strip off any at the beginning
+    if (path.startsWith("/")) {
+      // path starts with /, so append all other characters -if present
+      if (path.length() > 1) {
+        fullpath.append(path.substring(1));
+      }
+    } else {
+      fullpath.append(path);
+    }
+
+    //here there may be a trailing "/"
+    String finalpath = fullpath.toString();
+    if (finalpath.endsWith("/") && !"/".equals(finalpath)) {
+      finalpath = finalpath.substring(0, finalpath.length() - 1);
+
+    }
+    return finalpath;
+  }
+
+  /**
+   * split a path into elements, stripping empty elements
+   * @param path the path
+   * @return the split path
+   */
+  public static List<String> split(String path) {
+    //
+    String[] pathelements = path.split("/");
+    List<String> dirs = new ArrayList<String>(pathelements.length);
+    for (String pathelement : pathelements) {
+      if (!pathelement.isEmpty()) {
+        dirs.add(pathelement);
+      }
+    }
+    return dirs;
+  }
+
+  /**
+   * Get the last entry in a path; for an empty path
+   * returns "". The split logic is that of
+   * {@link #split(String)}
+   * @param path path of operation
+   * @return the last path entry or "" if none.
+   */
+  public static String lastPathEntry(String path) {
+    List<String> splits = split(path);
+    if (splits.isEmpty()) {
+      // empty path. Return ""
+      return "";
+    } else {
+      return splits.get(splits.size() - 1);
+    }
+  }
+
+  /**
+   * Get the parent of a path
+   * @param path path to look at
+   * @return the parent path
+   * @throws PathNotFoundException if the path was at root.
+   */
+  public static String parentOf(String path) throws PathNotFoundException {
+    List<String> elements = split(path);
+
+    int size = elements.size();
+    if (size == 0) {
+      throw new PathNotFoundException("No parent of " + path);
+    }
+    if (size == 1) {
+      return "/";
+    }
+    elements.remove(size - 1);
+    StringBuilder parent = new StringBuilder(path.length());
+    for (String element : elements) {
+      parent.append("/");
+      parent.append(element);
+    }
+    return parent.toString();
+  }
+
+  /**
+   * Perform any formatting for the registry needed to convert
+   * non-simple-DNS elements
+   * @param element element to encode
+   * @return an encoded string
+   */
+  public static String encodeForRegistry(String element) {
+    return IDN.toASCII(element);
+  }
+
+  /**
+   * Perform whatever transforms are needed to get a YARN ID into
+   * a DNS-compatible name
+   * @param yarnId ID as string of YARN application, instance or container
+   * @return a string suitable for use in registry paths.
+   */
+  public static String encodeYarnID(String yarnId) {
+    return yarnId.replace("_", "-");
+  }
+}


[5/6] YARN-913 service registry: YARN-2652 add hadoop-yarn-registry package under hadoop-yarn

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryTypeUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryTypeUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryTypeUtils.java
new file mode 100644
index 0000000..b4254a3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryTypeUtils.java
@@ -0,0 +1,240 @@
+/*
+ * 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.hadoop.registry.client.binding;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.registry.client.types.AddressTypes;
+import org.apache.hadoop.registry.client.types.Endpoint;
+import org.apache.hadoop.registry.client.types.ProtocolTypes;
+
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Static methods to work with registry types —primarily endpoints and the
+ * list representation of addresses.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RegistryTypeUtils {
+
+  /**
+   * Create a URL endpoint from a list of URIs
+   * @param api implemented API
+   * @param protocolType protocol type
+   * @param uris URIs
+   * @return a new endpoint
+   */
+  public static Endpoint urlEndpoint(String api,
+      String protocolType,
+      URI... uris) {
+    return new Endpoint(api, protocolType, uris);
+  }
+
+  /**
+   * Create a REST endpoint from a list of URIs
+   * @param api implemented API
+   * @param uris URIs
+   * @return a new endpoint
+   */
+  public static Endpoint restEndpoint(String api,
+      URI... uris) {
+    return urlEndpoint(api, ProtocolTypes.PROTOCOL_REST, uris);
+  }
+
+  /**
+   * Create a Web UI endpoint from a list of URIs
+   * @param api implemented API
+   * @param uris URIs
+   * @return a new endpoint
+   */
+  public static Endpoint webEndpoint(String api,
+      URI... uris) {
+    return urlEndpoint(api, ProtocolTypes.PROTOCOL_WEBUI, uris);
+  }
+
+  /**
+   * Create an internet address endpoint from a list of URIs
+   * @param api implemented API
+   * @param protocolType protocol type
+   * @param hostname hostname/FQDN
+   * @param port port
+   * @return a new endpoint
+   */
+
+  public static Endpoint inetAddrEndpoint(String api,
+      String protocolType,
+      String hostname,
+      int port) {
+    Preconditions.checkArgument(api != null, "null API");
+    Preconditions.checkArgument(protocolType != null, "null protocolType");
+    Preconditions.checkArgument(hostname != null, "null hostname");
+    return new Endpoint(api,
+        AddressTypes.ADDRESS_HOSTNAME_AND_PORT,
+        protocolType,
+        tuplelist(hostname, Integer.toString(port)));
+  }
+
+  /**
+   * Create an IPC endpoint
+   * @param api API
+   * @param protobuf flag to indicate whether or not the IPC uses protocol
+   * buffers
+   * @param address the address as a tuple of (hostname, port)
+   * @return the new endpoint
+   */
+  public static Endpoint ipcEndpoint(String api,
+      boolean protobuf, List<String> address) {
+    ArrayList<List<String>> addressList = new ArrayList<List<String>>();
+    if (address != null) {
+      addressList.add(address);
+    }
+    return new Endpoint(api,
+        AddressTypes.ADDRESS_HOSTNAME_AND_PORT,
+        protobuf ? ProtocolTypes.PROTOCOL_HADOOP_IPC_PROTOBUF
+                 : ProtocolTypes.PROTOCOL_HADOOP_IPC,
+        addressList);
+  }
+
+  /**
+   * Create a single-element list of tuples from the input.
+   * that is, an input ("a","b","c") is converted into a list
+   * in the form [["a","b","c"]]
+   * @param t1 tuple elements
+   * @return a list containing a single tuple
+   */
+  public static List<List<String>> tuplelist(String... t1) {
+    List<List<String>> outer = new ArrayList<List<String>>();
+    outer.add(tuple(t1));
+    return outer;
+  }
+
+  /**
+   * Create a tuples from the input.
+   * that is, an input ("a","b","c") is converted into a list
+   * in the form ["a","b","c"]
+   * @param t1 tuple elements
+   * @return a single tuple as a list
+   */
+  public static List<String> tuple(String... t1) {
+    return Arrays.asList(t1);
+  }
+
+  /**
+   * Create a tuples from the input, converting all to Strings in the process
+   * that is, an input ("a", 7, true) is converted into a list
+   * in the form ["a","7,"true"]
+   * @param t1 tuple elements
+   * @return a single tuple as a list
+   */
+  public static List<String> tuple(Object... t1) {
+    List<String> l = new ArrayList<String>(t1.length);
+    for (Object t : t1) {
+      l.add(t.toString());
+    }
+    return l;
+  }
+
+  /**
+   * Convert a socket address pair into a string tuple, (host, port).
+   * TODO JDK7: move to InetAddress.getHostString() to avoid DNS lookups.
+   * @param address an address
+   * @return an element for the address list
+   */
+  public static List<String> marshall(InetSocketAddress address) {
+    return tuple(address.getHostName(), address.getPort());
+  }
+
+  /**
+   * Require a specific address type on an endpoint
+   * @param required required type
+   * @param epr endpoint
+   * @throws InvalidRecordException if the type is wrong
+   */
+  public static void requireAddressType(String required, Endpoint epr) throws
+      InvalidRecordException {
+    if (!required.equals(epr.addressType)) {
+      throw new InvalidRecordException(
+          epr.toString(),
+          "Address type of " + epr.addressType
+          + " does not match required type of "
+          + required);
+    }
+  }
+
+  /**
+   * Get a single URI endpoint
+   * @param epr endpoint
+   * @return the uri of the first entry in the address list. Null if the endpoint
+   * itself is null
+   * @throws InvalidRecordException if the type is wrong, there are no addresses
+   * or the payload ill-formatted
+   */
+  public static List<String> retrieveAddressesUriType(Endpoint epr)
+      throws InvalidRecordException {
+    if (epr == null) {
+      return null;
+    }
+    requireAddressType(AddressTypes.ADDRESS_URI, epr);
+    List<List<String>> addresses = epr.addresses;
+    if (addresses.size() < 1) {
+      throw new InvalidRecordException(epr.toString(),
+          "No addresses in endpoint");
+    }
+    List<String> results = new ArrayList<String>(addresses.size());
+    for (List<String> address : addresses) {
+      if (address.size() != 1) {
+        throw new InvalidRecordException(epr.toString(),
+            "Address payload invalid: wrong element count: " +
+            address.size());
+      }
+      results.add(address.get(0));
+    }
+    return results;
+  }
+
+  /**
+   * Get the address URLs. Guranteed to return at least one address.
+   * @param epr endpoint
+   * @return the address as a URL
+   * @throws InvalidRecordException if the type is wrong, there are no addresses
+   * or the payload ill-formatted
+   * @throws MalformedURLException address can't be turned into a URL
+   */
+  public static List<URL> retrieveAddressURLs(Endpoint epr)
+      throws InvalidRecordException, MalformedURLException {
+    if (epr == null) {
+      throw new InvalidRecordException("", "Null endpoint");
+    }
+    List<String> addresses = retrieveAddressesUriType(epr);
+    List<URL> results = new ArrayList<URL>(addresses.size());
+    for (String address : addresses) {
+      results.add(new URL(address));
+    }
+    return results;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
new file mode 100644
index 0000000..3b28a02
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
@@ -0,0 +1,362 @@
+/*
+ * 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.hadoop.registry.client.binding;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.exceptions.InvalidPathnameException;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.registry.client.exceptions.NoRecordException;
+import org.apache.hadoop.registry.client.impl.zk.RegistryInternalConstants;
+import org.apache.hadoop.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.client.types.ServiceRecordHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.registry.client.binding.RegistryPathUtils.*;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Utility methods for working with a registry.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RegistryUtils {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RegistryUtils.class);
+
+  /**
+   * Buld the user path -switches to the system path if the user is "".
+   * It also cross-converts the username to ascii via punycode
+   * @param shortname username or ""
+   * @return the path to the user
+   */
+  public static String homePathForUser(String shortname) {
+    Preconditions.checkArgument(shortname != null, "null user");
+
+    // catch recursion
+    if (shortname.startsWith(RegistryConstants.PATH_USERS)) {
+      return shortname;
+    }
+    if (shortname.isEmpty()) {
+      return RegistryConstants.PATH_SYSTEM_SERVICES;
+    }
+    return RegistryPathUtils.join(RegistryConstants.PATH_USERS,
+        encodeForRegistry(shortname));
+  }
+
+  /**
+   * Create a service classpath
+   * @param user username or ""
+   * @param serviceClass service name
+   * @return a full path
+   */
+  public static String serviceclassPath(String user,
+      String serviceClass) {
+    String services = join(homePathForUser(user),
+        RegistryConstants.PATH_USER_SERVICES);
+    return join(services,
+        serviceClass);
+  }
+
+  /**
+   * Create a path to a service under a user & service class
+   * @param user username or ""
+   * @param serviceClass service name
+   * @param serviceName service name unique for that user & service class
+   * @return a full path
+   */
+  public static String servicePath(String user,
+      String serviceClass,
+      String serviceName) {
+
+    return join(
+        serviceclassPath(user, serviceClass),
+        serviceName);
+  }
+
+  /**
+   * Create a path for listing components under a service
+   * @param user username or ""
+   * @param serviceClass service name
+   * @param serviceName service name unique for that user & service class
+   * @return a full path
+   */
+  public static String componentListPath(String user,
+      String serviceClass, String serviceName) {
+
+    return join(servicePath(user, serviceClass, serviceName),
+        RegistryConstants.SUBPATH_COMPONENTS);
+  }
+
+  /**
+   * Create the path to a service record for a component
+   * @param user username or ""
+   * @param serviceClass service name
+   * @param serviceName service name unique for that user & service class
+   * @param componentName unique name/ID of the component
+   * @return a full path
+   */
+  public static String componentPath(String user,
+      String serviceClass, String serviceName, String componentName) {
+
+    return join(
+        componentListPath(user, serviceClass, serviceName),
+        componentName);
+  }
+
+  /**
+   * List service records directly under a path
+   * @param registryOperations registry operations instance
+   * @param path path to list
+   * @return a mapping of the service records that were resolved, indexed
+   * by their full path
+   * @throws IOException
+   */
+  public static Map<String, ServiceRecord> listServiceRecords(
+      RegistryOperations registryOperations,
+      String path) throws IOException {
+    Map<String, RegistryPathStatus> children =
+        statChildren(registryOperations, path);
+    return extractServiceRecords(registryOperations,
+        path,
+        children.values());
+  }
+
+  /**
+   * List children of a directory and retrieve their
+   * {@link RegistryPathStatus} values.
+   * <p>
+   * This is not an atomic operation; A child may be deleted
+   * during the iteration through the child entries. If this happens,
+   * the <code>PathNotFoundException</code> is caught and that child
+   * entry ommitted.
+   *
+   * @param path path
+   * @return a possibly empty map of child entries listed by
+   * their short name.
+   * @throws PathNotFoundException path is not in the registry.
+   * @throws InvalidPathnameException the path is invalid.
+   * @throws IOException Any other IO Exception
+   */
+  public static Map<String, RegistryPathStatus> statChildren(
+      RegistryOperations registryOperations,
+      String path)
+      throws PathNotFoundException,
+      InvalidPathnameException,
+      IOException {
+    List<String> childNames = registryOperations.list(path);
+    Map<String, RegistryPathStatus> results =
+        new HashMap<String, RegistryPathStatus>();
+    for (String childName : childNames) {
+      String child = join(path, childName);
+      try {
+        RegistryPathStatus stat = registryOperations.stat(child);
+        results.put(childName, stat);
+      } catch (PathNotFoundException pnfe) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("stat failed on {}: moved? {}", child, pnfe, pnfe);
+        }
+        // and continue
+      }
+    }
+    return results;
+  }
+
+  /**
+   * Get the home path of the current user.
+   * <p>
+   *  In an insecure cluster, the environment variable
+   *  <code>HADOOP_USER_NAME</code> is queried <i>first</i>.
+   * <p>
+   * This means that in a YARN container where the creator set this
+   * environment variable to propagate their identity, the defined
+   * user name is used in preference to the actual user.
+   * <p>
+   * In a secure cluster, the kerberos identity of the current user is used.
+   * @return a path for the current user's home dir.
+   * @throws RuntimeException if the current user identity cannot be determined
+   * from the OS/kerberos.
+   */
+  public static String homePathForCurrentUser() {
+    String shortUserName = currentUsernameUnencoded();
+    return homePathForUser(shortUserName);
+  }
+
+  /**
+   * Get the current username, before any encoding has been applied.
+   * @return the current user from the kerberos identity, falling back
+   * to the user and/or env variables.
+   */
+  private static String currentUsernameUnencoded() {
+    String env_hadoop_username = System.getenv(
+        RegistryInternalConstants.HADOOP_USER_NAME);
+    return getCurrentUsernameUnencoded(env_hadoop_username);
+  }
+
+  /**
+   * Get the current username, using the value of the parameter
+   * <code>env_hadoop_username</code> if it is set on an insecure cluster.
+   * This ensures that the username propagates correctly across processes
+   * started by YARN.
+   * <p>
+   * This method is primarly made visible for testing.
+   * @param env_hadoop_username the environment variable
+   * @return the selected username
+   * @throws RuntimeException if there is a problem getting the short user
+   * name of the current user.
+   */
+  @VisibleForTesting
+  public static String getCurrentUsernameUnencoded(String env_hadoop_username) {
+    String shortUserName = null;
+    if (!UserGroupInformation.isSecurityEnabled()) {
+      shortUserName = env_hadoop_username;
+    }
+    if (StringUtils.isEmpty(shortUserName)) {
+      try {
+        shortUserName = UserGroupInformation.getCurrentUser().getShortUserName();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    return shortUserName;
+  }
+
+  /**
+   * Get the current user path formatted for the registry
+   * <p>
+   *  In an insecure cluster, the environment variable
+   *  <code>HADOOP_USER_NAME </code> is queried <i>first</i>.
+   * <p>
+   * This means that in a YARN container where the creator set this
+   * environment variable to propagate their identity, the defined
+   * user name is used in preference to the actual user.
+   * <p>
+   * In a secure cluster, the kerberos identity of the current user is used.
+   * @return the encoded shortname of the current user
+   * @throws RuntimeException if the current user identity cannot be determined
+   * from the OS/kerberos.
+   *
+   */
+  public static String currentUser() {
+    String shortUserName = currentUsernameUnencoded();
+    return encodeForRegistry(shortUserName);
+  }
+
+  /**
+   * Extract all service records under a list of stat operations...this
+   * skips entries that are too short or simply not matching
+   * @param operations operation support for fetches
+   * @param parentpath path of the parent of all the entries
+   * @param stats Collection of stat results
+   * @return a possibly empty map of fullpath:record.
+   * @throws IOException for any IO Operation that wasn't ignored.
+   */
+  public static Map<String, ServiceRecord> extractServiceRecords(
+      RegistryOperations operations,
+      String parentpath,
+      Collection<RegistryPathStatus> stats) throws IOException {
+    Map<String, ServiceRecord> results = new HashMap<String, ServiceRecord>(stats.size());
+    for (RegistryPathStatus stat : stats) {
+      if (stat.size > ServiceRecordHeader.getLength()) {
+        // maybe has data
+        String path = join(parentpath, stat.path);
+        try {
+          ServiceRecord serviceRecord = operations.resolve(path);
+          results.put(path, serviceRecord);
+        } catch (EOFException ignored) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("data too short for {}", path);
+          }
+        } catch (InvalidRecordException record) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Invalid record at {}", path);
+          }
+        } catch (NoRecordException record) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("No record at {}", path);
+          }
+        }
+      }
+    }
+    return results;
+  }
+
+  /**
+   * Extract all service records under a list of stat operations...this
+   * non-atomic action skips entries that are too short or simply not matching.
+   * <p>
+   * @param operations operation support for fetches
+   * @param parentpath path of the parent of all the entries
+   * @param stats a map of name:value mappings.
+   * @return a possibly empty map of fullpath:record.
+   * @throws IOException for any IO Operation that wasn't ignored.
+   */
+  public static Map<String, ServiceRecord> extractServiceRecords(
+      RegistryOperations operations,
+      String parentpath,
+      Map<String , RegistryPathStatus> stats) throws IOException {
+    return extractServiceRecords(operations, parentpath, stats.values());
+  }
+
+
+  /**
+   * Extract all service records under a list of stat operations...this
+   * non-atomic action skips entries that are too short or simply not matching.
+   * <p>
+   * @param operations operation support for fetches
+   * @param parentpath path of the parent of all the entries
+   * @param stats a map of name:value mappings.
+   * @return a possibly empty map of fullpath:record.
+   * @throws IOException for any IO Operation that wasn't ignored.
+   */
+  public static Map<String, ServiceRecord> extractServiceRecords(
+      RegistryOperations operations,
+      String parentpath) throws IOException {
+    return
+    extractServiceRecords(operations,
+        parentpath,
+        statChildren(operations, parentpath).values());
+  }
+
+
+
+  /**
+   * Static instance of service record marshalling
+   */
+  public static class ServiceRecordMarshal extends JsonSerDeser<ServiceRecord> {
+    public ServiceRecordMarshal() {
+      super(ServiceRecord.class, ServiceRecordHeader.getData());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/package-info.java
new file mode 100644
index 0000000..f99aa71
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Registry binding utility classes.
+ */
+package org.apache.hadoop.registry.client.binding;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/AuthenticationFailedException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/AuthenticationFailedException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/AuthenticationFailedException.java
new file mode 100644
index 0000000..aadb7fc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/AuthenticationFailedException.java
@@ -0,0 +1,39 @@
+/*
+ * 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.hadoop.registry.client.exceptions;
+
+/**
+ * Exception raised when client access wasn't authenticated.
+ * That is: the credentials provided were incomplete or invalid.
+ */
+public class AuthenticationFailedException extends RegistryIOException {
+  public AuthenticationFailedException(String path, Throwable cause) {
+    super(path, cause);
+  }
+
+  public AuthenticationFailedException(String path, String error) {
+    super(path, error);
+  }
+
+  public AuthenticationFailedException(String path,
+      String error,
+      Throwable cause) {
+    super(path, error, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/InvalidPathnameException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/InvalidPathnameException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/InvalidPathnameException.java
new file mode 100644
index 0000000..c984f41
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/InvalidPathnameException.java
@@ -0,0 +1,40 @@
+/*
+ * 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.hadoop.registry.client.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A path name was invalid. This is raised when a path string has
+ * characters in it that are not permitted.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class InvalidPathnameException extends RegistryIOException {
+  public InvalidPathnameException(String path, String message) {
+    super(path, message);
+  }
+
+  public InvalidPathnameException(String path,
+      String message,
+      Throwable cause) {
+    super(path, message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/InvalidRecordException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/InvalidRecordException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/InvalidRecordException.java
new file mode 100644
index 0000000..e4f545e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/InvalidRecordException.java
@@ -0,0 +1,41 @@
+/*
+ * 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.hadoop.registry.client.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Raised if an attempt to parse a record failed.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class InvalidRecordException extends RegistryIOException {
+
+  public InvalidRecordException(String path, String error) {
+    super(path, error);
+  }
+
+  public InvalidRecordException(String path,
+      String error,
+      Throwable cause) {
+    super(path, error, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoChildrenForEphemeralsException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoChildrenForEphemeralsException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoChildrenForEphemeralsException.java
new file mode 100644
index 0000000..24070a5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoChildrenForEphemeralsException.java
@@ -0,0 +1,48 @@
+/*
+ * 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.hadoop.registry.client.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This is a manifestation of the Zookeeper restrictions about
+ * what nodes may act as parents.
+ *
+ * Children are not allowed under ephemeral nodes. This is an aspect
+ * of ZK which isn't directly exposed to the registry API. It may
+ * surface if the registry is manipulated outside of the registry API.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class NoChildrenForEphemeralsException extends RegistryIOException {
+  public NoChildrenForEphemeralsException(String path, Throwable cause) {
+    super(path, cause);
+  }
+
+  public NoChildrenForEphemeralsException(String path, String error) {
+    super(path, error);
+  }
+
+  public NoChildrenForEphemeralsException(String path,
+      String error,
+      Throwable cause) {
+    super(path, error, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoPathPermissionsException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoPathPermissionsException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoPathPermissionsException.java
new file mode 100644
index 0000000..ce84f5b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoPathPermissionsException.java
@@ -0,0 +1,45 @@
+/*
+ * 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.hadoop.registry.client.exceptions;
+
+import org.apache.hadoop.fs.PathIOException;
+
+/**
+ * Raised on path permission exceptions.
+ * <p>
+ * This is similar to PathIOException, except that exception doesn't let
+ */
+public class NoPathPermissionsException extends RegistryIOException {
+  public NoPathPermissionsException(String path, Throwable cause) {
+    super(path, cause);
+  }
+
+  public NoPathPermissionsException(String path, String error) {
+    super(path, error);
+  }
+
+  public NoPathPermissionsException(String path, String error, Throwable cause) {
+    super(path, error, cause);
+  }
+
+  public NoPathPermissionsException(String message,
+      PathIOException cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoRecordException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoRecordException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoRecordException.java
new file mode 100644
index 0000000..160433f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/NoRecordException.java
@@ -0,0 +1,51 @@
+/*
+ * 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.hadoop.registry.client.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.client.types.ServiceRecordHeader;
+
+/**
+ * Raised if there is no {@link ServiceRecord} resolved at the end
+ * of the specified path, for reasons such as:
+ * <ul>
+ *   <li>There wasn't enough data to contain a Service Record.</li>
+ *   <li>The start of the data did not match the {@link ServiceRecordHeader}
+ *   header.</li>
+ * </ul>
+ *
+ * There may be valid data of some form at the end of the path, but it does
+ * not appear to be a Service Record.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class NoRecordException extends RegistryIOException {
+
+  public NoRecordException(String path, String error) {
+    super(path, error);
+  }
+
+  public NoRecordException(String path,
+      String error,
+      Throwable cause) {
+    super(path, error, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/RegistryIOException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/RegistryIOException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/RegistryIOException.java
new file mode 100644
index 0000000..ca966db
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/RegistryIOException.java
@@ -0,0 +1,58 @@
+/*
+ * 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.hadoop.registry.client.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.PathIOException;
+
+/**
+ * Base exception for registry operations.
+ * <p>
+ * These exceptions include the path of the failing operation wherever possible;
+ * this can be retrieved via {@link PathIOException#getPath()}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RegistryIOException extends PathIOException {
+
+  /**
+   * Build an exception from any other Path IO Exception.
+   * This propagates the path of the original exception
+   * @param message more specific text
+   * @param cause cause
+   */
+  public RegistryIOException(String message, PathIOException cause) {
+    super(cause.getPath() != null ? cause.getPath().toString() : "",
+        message,
+        cause);
+  }
+
+  public RegistryIOException(String path, Throwable cause) {
+    super(path, cause);
+  }
+
+  public RegistryIOException(String path, String error) {
+    super(path, error);
+  }
+
+  public RegistryIOException(String path, String error, Throwable cause) {
+    super(path, error, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/package-info.java
new file mode 100644
index 0000000..7d9c8ad
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/exceptions/package-info.java
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+/**
+ * Registry Service Exceptions
+ * <p>
+ * These are the Registry-specific exceptions that may be raised during
+ * Registry operations.
+ * <p>
+ * Other exceptions may be raised, especially <code>IOExceptions</code>
+ * triggered by network problems, and <code>IllegalArgumentException</code>
+ * exceptions that may be raised if invalid (often null) arguments are passed
+ * to a method call.
+ * <p>
+ *   All exceptions in this package are derived from
+ *   {@link org.apache.hadoop.registry.client.exceptions.RegistryIOException}
+ */
+package org.apache.hadoop.registry.client.exceptions;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java
new file mode 100644
index 0000000..db03936
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java
@@ -0,0 +1,55 @@
+/*
+ * 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.hadoop.registry.client.impl;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.registry.client.impl.zk.RegistryBindingSource;
+import org.apache.hadoop.registry.client.impl.zk.RegistryOperationsService;
+
+
+/**
+ * This is the client service for applications to work with the registry.
+ *
+ * It does not set up the root paths for the registry, is bonded
+ * to a user, and can be set to use SASL, anonymous or id:pass auth.
+ *
+ * For SASL, the client must be operating in the context of an authed user.
+ *
+ * For id:pass the client must have the relevant id & password, SASL is
+ * not used even if the client has credentials.
+ *
+ * For anonymous, nothing is used.
+ *
+ * Any SASL-authed client also has the ability to add one or more authentication
+ * id:pass pair on all future writes, and to reset them later.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RegistryOperationsClient extends RegistryOperationsService {
+
+  public RegistryOperationsClient(String name) {
+    super(name);
+  }
+
+  public RegistryOperationsClient(String name,
+      RegistryBindingSource bindingSource) {
+    super(name, bindingSource);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/package-info.java
new file mode 100644
index 0000000..d85b6a7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+/**
+ * Registry client services
+ * <p>
+ * These are classes which follow the YARN lifecycle and which implement
+ * the {@link org.apache.hadoop.registry.client.api.RegistryOperations}
+ * API.
+ */
+package org.apache.hadoop.registry.client.impl;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/BindingInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/BindingInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/BindingInformation.java
new file mode 100644
index 0000000..8ae003d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/BindingInformation.java
@@ -0,0 +1,41 @@
+/*
+ * 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.hadoop.registry.client.impl.zk;
+
+import org.apache.curator.ensemble.EnsembleProvider;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Binding information provided by a {@link RegistryBindingSource}
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class BindingInformation {
+
+  /**
+   * The Curator Ensemble Provider
+   */
+  public EnsembleProvider ensembleProvider;
+
+  /**
+   * Any information that may be useful for diagnostics
+   */
+  public String description;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/CuratorService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/CuratorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/CuratorService.java
new file mode 100644
index 0000000..a0e6365
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/CuratorService.java
@@ -0,0 +1,769 @@
+/*
+ * 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.hadoop.registry.client.impl.zk;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.curator.ensemble.EnsembleProvider;
+import org.apache.curator.ensemble.fixed.FixedEnsembleProvider;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CreateBuilder;
+import org.apache.curator.framework.api.DeleteBuilder;
+import org.apache.curator.framework.api.GetChildrenBuilder;
+import org.apache.curator.retry.BoundedExponentialBackoffRetry;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.service.ServiceStateException;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.exceptions.AuthenticationFailedException;
+import org.apache.hadoop.registry.client.exceptions.NoChildrenForEphemeralsException;
+import org.apache.hadoop.registry.client.exceptions.NoPathPermissionsException;
+import org.apache.hadoop.registry.client.exceptions.RegistryIOException;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * This service binds to Zookeeper via Apache Curator. It is more
+ * generic than just the YARN service registry; it does not implement
+ * any of the Registry Operations API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class CuratorService extends CompositeService
+    implements RegistryConstants, RegistryBindingSource {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CuratorService.class);
+
+  /**
+   * the Curator binding
+   */
+  private CuratorFramework curator;
+
+  /**
+   * Path to the registry root
+   */
+  private String registryRoot;
+
+  /**
+   * Supplied binding source. This defaults to being this
+   * service itself.
+   */
+  private final RegistryBindingSource bindingSource;
+
+  /**
+   * Security service
+   */
+  private RegistrySecurity registrySecurity;
+
+  /**
+   * the connection binding text for messages
+   */
+  private String connectionDescription;
+
+  /**
+   * Security connection diagnostics
+   */
+  private String securityConnectionDiagnostics = "";
+
+  /**
+   * Provider of curator "ensemble"; offers a basis for
+   * more flexible bonding in future.
+   */
+  private EnsembleProvider ensembleProvider;
+
+  /**
+   * Construct the service.
+   * @param name service name
+   * @param bindingSource source of binding information.
+   * If null: use this instance
+   */
+  public CuratorService(String name, RegistryBindingSource bindingSource) {
+    super(name);
+    if (bindingSource != null) {
+      this.bindingSource = bindingSource;
+    } else {
+      this.bindingSource = this;
+    }
+  }
+
+  /**
+   * Create an instance using this service as the binding source (i.e. read
+   * configuration options from the registry)
+   * @param name service name
+   */
+  public CuratorService(String name) {
+    this(name, null);
+  }
+
+  /**
+   * Init the service.
+   * This is where the security bindings are set up
+   * @param conf configuration of the service
+   * @throws Exception
+   */
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+
+    registryRoot = conf.getTrimmed(KEY_REGISTRY_ZK_ROOT,
+        DEFAULT_ZK_REGISTRY_ROOT);
+
+    // create and add the registy service
+    registrySecurity = new RegistrySecurity("registry security");
+    addService(registrySecurity);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Creating Registry with root {}", registryRoot);
+    }
+
+    super.serviceInit(conf);
+  }
+
+  /**
+   * Start the service.
+   * This is where the curator instance is started.
+   * @throws Exception
+   */
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+
+    // create the curator; rely on the registry security code
+    // to set up the JVM context and curator
+    curator = createCurator();
+  }
+
+  /**
+   * Close the ZK connection if it is open
+   */
+  @Override
+  protected void serviceStop() throws Exception {
+    IOUtils.closeStream(curator);
+    super.serviceStop();
+  }
+
+  /**
+   * Internal check that a service is in the live state
+   * @throws ServiceStateException if not
+   */
+  private void checkServiceLive() throws ServiceStateException {
+    if (!isInState(STATE.STARTED)) {
+      throw new ServiceStateException(
+          "Service " + getName() + " is in wrong state: "
+          + getServiceState());
+    }
+  }
+
+  /**
+   * Flag to indicate whether or not the registry is secure.
+   * Valid once the service is inited.
+   * @return service security policy
+   */
+  public boolean isSecure() {
+    return registrySecurity.isSecureRegistry();
+  }
+
+  /**
+   * Get the registry security helper
+   * @return the registry security helper
+   */
+  protected RegistrySecurity getRegistrySecurity() {
+    return registrySecurity;
+  }
+
+  /**
+   * Build the security diagnostics string
+   * @return a string for diagnostics
+   */
+  protected String buildSecurityDiagnostics() {
+    // build up the security connection diags
+    if (!isSecure()) {
+      return "security disabled";
+    } else {
+      StringBuilder builder = new StringBuilder();
+      builder.append("secure cluster; ");
+      builder.append(registrySecurity.buildSecurityDiagnostics());
+      return builder.toString();
+    }
+  }
+
+  /**
+   * Create a new curator instance off the root path; using configuration
+   * options provided in the service configuration to set timeouts and
+   * retry policy.
+   * @return the newly created creator
+   */
+  private CuratorFramework createCurator() throws IOException {
+    Configuration conf = getConfig();
+    createEnsembleProvider();
+    int sessionTimeout = conf.getInt(KEY_REGISTRY_ZK_SESSION_TIMEOUT,
+        DEFAULT_ZK_SESSION_TIMEOUT);
+    int connectionTimeout = conf.getInt(KEY_REGISTRY_ZK_CONNECTION_TIMEOUT,
+        DEFAULT_ZK_CONNECTION_TIMEOUT);
+    int retryTimes = conf.getInt(KEY_REGISTRY_ZK_RETRY_TIMES,
+        DEFAULT_ZK_RETRY_TIMES);
+    int retryInterval = conf.getInt(KEY_REGISTRY_ZK_RETRY_INTERVAL,
+        DEFAULT_ZK_RETRY_INTERVAL);
+    int retryCeiling = conf.getInt(KEY_REGISTRY_ZK_RETRY_CEILING,
+        DEFAULT_ZK_RETRY_CEILING);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Creating CuratorService with connection {}",
+          connectionDescription);
+    }
+    CuratorFramework framework;
+
+    synchronized (CuratorService.class) {
+      // set the security options
+
+      //log them
+      securityConnectionDiagnostics = buildSecurityDiagnostics();
+
+      // build up the curator itself
+      CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
+      builder.ensembleProvider(ensembleProvider)
+       .connectionTimeoutMs(connectionTimeout)
+       .sessionTimeoutMs(sessionTimeout)
+
+       .retryPolicy(new BoundedExponentialBackoffRetry(retryInterval,
+           retryCeiling,
+           retryTimes));
+
+      // set up the builder AND any JVM context
+      registrySecurity.applySecurityEnvironment(builder);
+
+      framework = builder.build();
+      framework.start();
+    }
+
+    return framework;
+  }
+
+  @Override
+  public String toString() {
+    return super.toString()
+           + bindingDiagnosticDetails();
+  }
+
+  /**
+   * Get the binding diagnostics
+   * @return a diagnostics string valid after the service is started.
+   */
+  public String bindingDiagnosticDetails() {
+    return " Connection=\"" + connectionDescription + "\""
+           + " root=\"" + registryRoot + "\""
+           + " " + securityConnectionDiagnostics;
+  }
+
+  /**
+   * Create a full path from the registry root and the supplied subdir
+   * @param path path of operation
+   * @return an absolute path
+   * @throws IllegalArgumentException if the path is invalide
+   */
+  protected String createFullPath(String path) throws IOException {
+    return RegistryPathUtils.createFullPath(registryRoot, path);
+  }
+
+  /**
+   * Get the registry binding source ... this can be used to
+   * create new ensemble providers
+   * @return the registry binding source in use
+   */
+  public RegistryBindingSource getBindingSource() {
+    return bindingSource;
+  }
+
+  /**
+   * Create the ensemble provider for this registry, by invoking
+   * {@link RegistryBindingSource#supplyBindingInformation()} on
+   * the provider stored in {@link #bindingSource}
+   * Sets {@link #ensembleProvider} to that value;
+   * sets {@link #connectionDescription} to the binding info
+   * for use in toString and logging;
+   *
+   */
+  protected void createEnsembleProvider() {
+    BindingInformation binding = bindingSource.supplyBindingInformation();
+    connectionDescription = binding.description
+                            + " " + securityConnectionDiagnostics;
+    ensembleProvider = binding.ensembleProvider;
+  }
+
+  /**
+   * Supply the binding information.
+   * This implementation returns a fixed ensemble bonded to
+   * the quorum supplied by {@link #buildConnectionString()}
+   * @return the binding information
+   */
+  @Override
+  public BindingInformation supplyBindingInformation() {
+    BindingInformation binding = new BindingInformation();
+    String connectString = buildConnectionString();
+    binding.ensembleProvider = new FixedEnsembleProvider(connectString);
+    binding.description =
+        "fixed ZK quorum \"" + connectString + "\"";
+    return binding;
+  }
+
+  /**
+   * Override point: get the connection string used to connect to
+   * the ZK service
+   * @return a registry quorum
+   */
+  protected String buildConnectionString() {
+    return getConfig().getTrimmed(KEY_REGISTRY_ZK_QUORUM,
+        DEFAULT_REGISTRY_ZK_QUORUM);
+  }
+
+  /**
+   * Create an IOE when an operation fails
+   * @param path path of operation
+   * @param operation operation attempted
+   * @param exception caught the exception caught
+   * @return an IOE to throw that contains the path and operation details.
+   */
+  protected IOException operationFailure(String path,
+      String operation,
+      Exception exception) {
+    return operationFailure(path, operation, exception, null);
+  }
+
+  /**
+   * Create an IOE when an operation fails
+   * @param path path of operation
+   * @param operation operation attempted
+   * @param exception caught the exception caught
+   * @return an IOE to throw that contains the path and operation details.
+   */
+  protected IOException operationFailure(String path,
+      String operation,
+      Exception exception,
+      List<ACL> acls) {
+    IOException ioe;
+    String aclList = "[" + RegistrySecurity.aclsToString(acls) + "]";
+    if (exception instanceof KeeperException.NoNodeException) {
+      ioe = new PathNotFoundException(path);
+    } else if (exception instanceof KeeperException.NodeExistsException) {
+      ioe = new FileAlreadyExistsException(path);
+    } else if (exception instanceof KeeperException.NoAuthException) {
+      ioe = new NoPathPermissionsException(path,
+          "Not authorized to access path; ACLs: " + aclList);
+    } else if (exception instanceof KeeperException.NotEmptyException) {
+      ioe = new PathIsNotEmptyDirectoryException(path);
+    } else if (exception instanceof KeeperException.AuthFailedException) {
+      ioe = new AuthenticationFailedException(path,
+          "Authentication Failed: " + exception, exception);
+    } else if (exception instanceof KeeperException.NoChildrenForEphemeralsException) {
+      ioe = new NoChildrenForEphemeralsException(path,
+          "Cannot create a path under an ephemeral node: " + exception,
+          exception);
+    } else if (exception instanceof KeeperException.InvalidACLException) {
+      // this is a security exception of a kind
+      // include the ACLs to help the diagnostics
+      StringBuilder builder = new StringBuilder();
+      builder.append("Path access failure ").append(aclList);
+      builder.append(" ");
+      builder.append(securityConnectionDiagnostics);
+      ioe = new NoPathPermissionsException(path, builder.toString());
+    } else {
+      ioe = new RegistryIOException(path,
+          "Failure of " + operation + " on " + path + ": " +
+          exception.toString(),
+          exception);
+    }
+    if (ioe.getCause() == null) {
+      ioe.initCause(exception);
+    }
+    return ioe;
+  }
+
+  /**
+   * Create a path if it does not exist.
+   * The check is poll + create; there's a risk that another process
+   * may create the same path before the create() operation is executed/
+   * propagated to the ZK node polled.
+   *
+   * @param path path to create
+   * @param acl ACL for path -used when creating a new entry
+   * @param createParents flag to trigger parent creation
+   * @return true iff the path was created
+   * @throws IOException
+   */
+  @VisibleForTesting
+  public boolean maybeCreate(String path,
+      CreateMode mode,
+      List<ACL> acl,
+      boolean createParents) throws IOException {
+    return zkMkPath(path, mode, createParents, acl);
+  }
+
+  /**
+   * Stat the file
+   * @param path path of operation
+   * @return a curator stat entry
+   * @throws IOException on a failure
+   * @throws PathNotFoundException if the path was not found
+   */
+  public Stat zkStat(String path) throws IOException {
+    checkServiceLive();
+    String fullpath = createFullPath(path);
+    Stat stat;
+    try {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Stat {}", fullpath);
+      }
+      stat = curator.checkExists().forPath(fullpath);
+    } catch (Exception e) {
+      throw operationFailure(fullpath, "read()", e);
+    }
+    if (stat == null) {
+      throw new PathNotFoundException(path);
+    }
+    return stat;
+  }
+
+  /**
+   * Get the ACLs of a path
+   * @param path path of operation
+   * @return a possibly empty list of ACLs
+   * @throws IOException
+   */
+  public List<ACL> zkGetACLS(String path) throws IOException {
+    checkServiceLive();
+    String fullpath = createFullPath(path);
+    List<ACL> acls;
+    try {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("GetACLS {}", fullpath);
+      }
+      acls = curator.getACL().forPath(fullpath);
+    } catch (Exception e) {
+      throw operationFailure(fullpath, "read()", e);
+    }
+    if (acls == null) {
+      throw new PathNotFoundException(path);
+    }
+    return acls;
+  }
+
+  /**
+   * Probe for a path existing
+   * @param path path of operation
+   * @return true if the path was visible from the ZK server
+   * queried.
+   * @throws IOException on any exception other than
+   * {@link PathNotFoundException}
+   */
+  public boolean zkPathExists(String path) throws IOException {
+    checkServiceLive();
+    try {
+      return zkStat(path) != null;
+    } catch (PathNotFoundException e) {
+      return false;
+    } catch (IOException e) {
+      throw e;
+    }
+  }
+
+  /**
+   * Verify a path exists
+   * @param path path of operation
+   * @throws PathNotFoundException if the path is absent
+   * @throws IOException
+   */
+  public String zkPathMustExist(String path) throws IOException {
+    zkStat(path);
+    return path;
+  }
+
+  /**
+   * Create a directory. It is not an error if it already exists
+   * @param path path to create
+   * @param mode mode for path
+   * @param createParents flag to trigger parent creation
+   * @param acls ACL for path
+   * @throws IOException any problem
+   */
+  public boolean zkMkPath(String path,
+      CreateMode mode,
+      boolean createParents,
+      List<ACL> acls)
+      throws IOException {
+    checkServiceLive();
+    path = createFullPath(path);
+    if (acls == null || acls.isEmpty()) {
+      throw new NoPathPermissionsException(path, "Empty ACL list");
+    }
+
+    try {
+      RegistrySecurity.AclListInfo aclInfo =
+          new RegistrySecurity.AclListInfo(acls);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Creating path {} with mode {} and ACL {}",
+            path, mode, aclInfo);
+      }
+      CreateBuilder createBuilder = curator.create();
+      createBuilder.withMode(mode).withACL(acls);
+      if (createParents) {
+        createBuilder.creatingParentsIfNeeded();
+      }
+      createBuilder.forPath(path);
+
+    } catch (KeeperException.NodeExistsException e) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("path already present: {}", path, e);
+      }
+      return false;
+    } catch (Exception e) {
+      throw operationFailure(path, "mkdir() ", e, acls);
+    }
+    return true;
+  }
+
+  /**
+   * Recursively make a path
+   * @param path path to create
+   * @param acl ACL for path
+   * @throws IOException any problem
+   */
+  public void zkMkParentPath(String path,
+      List<ACL> acl) throws
+      IOException {
+    // split path into elements
+
+    zkMkPath(RegistryPathUtils.parentOf(path),
+        CreateMode.PERSISTENT, true, acl);
+  }
+
+  /**
+   * Create a path with given data. byte[0] is used for a path
+   * without data
+   * @param path path of operation
+   * @param data initial data
+   * @param acls
+   * @throws IOException
+   */
+  public void zkCreate(String path,
+      CreateMode mode,
+      byte[] data,
+      List<ACL> acls) throws IOException {
+    Preconditions.checkArgument(data != null, "null data");
+    checkServiceLive();
+    String fullpath = createFullPath(path);
+    try {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Creating {} with {} bytes of data and ACL {}",
+            fullpath, data.length,
+            new RegistrySecurity.AclListInfo(acls));
+      }
+      curator.create().withMode(mode).withACL(acls).forPath(fullpath, data);
+    } catch (Exception e) {
+      throw operationFailure(fullpath, "create()", e, acls);
+    }
+  }
+
+  /**
+   * Update the data for a path
+   * @param path path of operation
+   * @param data new data
+   * @throws IOException
+   */
+  public void zkUpdate(String path, byte[] data) throws IOException {
+    Preconditions.checkArgument(data != null, "null data");
+    checkServiceLive();
+    path = createFullPath(path);
+    try {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Updating {} with {} bytes", path, data.length);
+      }
+      curator.setData().forPath(path, data);
+    } catch (Exception e) {
+      throw operationFailure(path, "update()", e);
+    }
+  }
+
+  /**
+   * Create or update an entry
+   * @param path path
+   * @param data data
+   * @param acl ACL for path -used when creating a new entry
+   * @param overwrite enable overwrite
+   * @throws IOException
+   * @return true if the entry was created, false if it was simply updated.
+   */
+  public boolean zkSet(String path,
+      CreateMode mode,
+      byte[] data,
+      List<ACL> acl, boolean overwrite) throws IOException {
+    Preconditions.checkArgument(data != null, "null data");
+    checkServiceLive();
+    if (!zkPathExists(path)) {
+      zkCreate(path, mode, data, acl);
+      return true;
+    } else {
+      if (overwrite) {
+        zkUpdate(path, data);
+        return false;
+      } else {
+        throw new FileAlreadyExistsException(path);
+      }
+    }
+  }
+
+  /**
+   * Delete a directory/directory tree.
+   * It is not an error to delete a path that does not exist
+   * @param path path of operation
+   * @param recursive flag to trigger recursive deletion
+   * @param backgroundCallback callback; this being set converts the operation
+   * into an async/background operation.
+   * task
+   * @throws IOException on problems other than no-such-path
+   */
+  public void zkDelete(String path,
+      boolean recursive,
+      BackgroundCallback backgroundCallback) throws IOException {
+    checkServiceLive();
+    String fullpath = createFullPath(path);
+    try {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Deleting {}", fullpath);
+      }
+      DeleteBuilder delete = curator.delete();
+      if (recursive) {
+        delete.deletingChildrenIfNeeded();
+      }
+      if (backgroundCallback != null) {
+        delete.inBackground(backgroundCallback);
+      }
+      delete.forPath(fullpath);
+    } catch (KeeperException.NoNodeException e) {
+      // not an error
+    } catch (Exception e) {
+      throw operationFailure(fullpath, "delete()", e);
+    }
+  }
+
+  /**
+   * List all children of a path
+   * @param path path of operation
+   * @return a possibly empty list of children
+   * @throws IOException
+   */
+  public List<String> zkList(String path) throws IOException {
+    checkServiceLive();
+    String fullpath = createFullPath(path);
+    try {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("ls {}", fullpath);
+      }
+      GetChildrenBuilder builder = curator.getChildren();
+      List<String> children = builder.forPath(fullpath);
+      return children;
+    } catch (Exception e) {
+      throw operationFailure(path, "ls()", e);
+    }
+  }
+
+  /**
+   * Read data on a path
+   * @param path path of operation
+   * @return the data
+   * @throws IOException read failure
+   */
+  public byte[] zkRead(String path) throws IOException {
+    checkServiceLive();
+    String fullpath = createFullPath(path);
+    try {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Reading {}", fullpath);
+      }
+      return curator.getData().forPath(fullpath);
+    } catch (Exception e) {
+      throw operationFailure(fullpath, "read()", e);
+    }
+  }
+
+  /**
+   * Return a path dumper instance which can do a full dump
+   * of the registry tree in its <code>toString()</code>
+   * operation
+   * @return a class to dump the registry
+   * @param verbose verbose flag - includes more details (such as ACLs)
+   */
+  public ZKPathDumper dumpPath(boolean verbose) {
+    return new ZKPathDumper(curator, registryRoot, verbose);
+  }
+
+  /**
+   * Add a new write access entry for all future write operations.
+   * @param id ID to use
+   * @param pass password
+   * @throws IOException on any failure to build the digest
+   */
+  public boolean addWriteAccessor(String id, String pass) throws IOException {
+    RegistrySecurity security = getRegistrySecurity();
+    ACL digestACL = new ACL(ZooDefs.Perms.ALL,
+        security.toDigestId(security.digest(id, pass)));
+    return security.addDigestACL(digestACL);
+  }
+
+  /**
+   * Clear all write accessors
+   */
+  public void clearWriteAccessors() {
+    getRegistrySecurity().resetDigestACLs();
+  }
+
+
+  /**
+   * Diagnostics method to dump a registry robustly.
+   * Any exception raised is swallowed
+   * @param verbose verbose path dump
+   * @return the registry tree
+   */
+  protected String dumpRegistryRobustly(boolean verbose) {
+    try {
+      ZKPathDumper pathDumper = dumpPath(verbose);
+      return pathDumper.toString();
+    } catch (Exception e) {
+      // ignore
+      LOG.debug("Ignoring exception:  {}", e);
+    }
+    return "";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryBindingSource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryBindingSource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryBindingSource.java
new file mode 100644
index 0000000..bab4742
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryBindingSource.java
@@ -0,0 +1,36 @@
+/*
+ * 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.hadoop.registry.client.impl.zk;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Interface which can be implemented by a registry binding source
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface RegistryBindingSource {
+
+  /**
+   * Supply the binding information for this registry
+   * @return the binding information data
+   */
+  BindingInformation supplyBindingInformation();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryInternalConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryInternalConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryInternalConstants.java
new file mode 100644
index 0000000..f04673a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryInternalConstants.java
@@ -0,0 +1,81 @@
+/*
+ * 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.hadoop.registry.client.impl.zk;
+
+import org.apache.zookeeper.ZooDefs;
+
+/**
+ * Internal constants for the registry.
+ *
+ * These are the things which aren't visible to users.
+ *
+ */
+public interface RegistryInternalConstants {
+
+  /**
+   * Pattern of a single entry in the registry path. : {@value}.
+   * <p>
+   * This is what constitutes a valid hostname according to current RFCs.
+   * Alphanumeric first two and last one digit, alphanumeric
+   * and hyphens allowed in between.
+   * <p>
+   * No upper limit is placed on the size of an entry.
+   */
+  String VALID_PATH_ENTRY_PATTERN =
+      "([a-z0-9]|[a-z0-9][a-z0-9\\-]*[a-z0-9])";
+
+  /**
+   * Permissions for readers: {@value}.
+   */
+  int PERMISSIONS_REGISTRY_READERS = ZooDefs.Perms.READ;
+
+  /**
+   * Permissions for system services: {@value}
+   */
+  int PERMISSIONS_REGISTRY_SYSTEM_SERVICES = ZooDefs.Perms.ALL;
+
+  /**
+   * Permissions for a user's root entry: {@value}.
+   * All except the admin permissions (ACL access) on a node
+   */
+  int PERMISSIONS_REGISTRY_USER_ROOT =
+      ZooDefs.Perms.READ | ZooDefs.Perms.WRITE | ZooDefs.Perms.CREATE |
+      ZooDefs.Perms.DELETE;
+
+  /**
+   * Name of the SASL auth provider which has to be added to ZK server to enable
+   * sasl: auth patterns: {@value}.
+   *
+   * Without this callers can connect via SASL, but
+   * they can't use it in ACLs
+   */
+  String SASLAUTHENTICATION_PROVIDER =
+      "org.apache.zookeeper.server.auth.SASLAuthenticationProvider";
+
+  /**
+   * String to use as the prefix when declaring a new auth provider: {@value}.
+   */
+  String ZOOKEEPER_AUTH_PROVIDER = "zookeeper.authProvider";
+
+  /**
+   * This the Hadoop environment variable which propagates the identity
+   * of a user in an insecure cluster
+   */
+  String HADOOP_USER_NAME = "HADOOP_USER_NAME";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryOperationsService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryOperationsService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryOperationsService.java
new file mode 100644
index 0000000..c54c205
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryOperationsService.java
@@ -0,0 +1,155 @@
+/*
+ * 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.hadoop.registry.client.impl.zk;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.registry.client.api.BindFlags;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+
+import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.exceptions.InvalidPathnameException;
+import org.apache.hadoop.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * The Registry operations service.
+ * <p>
+ * This service implements the {@link RegistryOperations}
+ * API by mapping the commands to zookeeper operations, and translating
+ * results and exceptions back into those specified by the API.
+ * <p>
+ * Factory methods should hide the detail that this has been implemented via
+ * the {@link CuratorService} by returning it cast to that
+ * {@link RegistryOperations} interface, rather than this implementation class.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RegistryOperationsService extends CuratorService
+  implements RegistryOperations {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RegistryOperationsService.class);
+
+  private final RegistryUtils.ServiceRecordMarshal serviceRecordMarshal
+      = new RegistryUtils.ServiceRecordMarshal();
+
+  public RegistryOperationsService(String name) {
+    this(name, null);
+  }
+
+  public RegistryOperationsService() {
+    this("RegistryOperationsService");
+  }
+
+  public RegistryOperationsService(String name,
+      RegistryBindingSource bindingSource) {
+    super(name, bindingSource);
+  }
+
+  /**
+   * Get the aggregate set of ACLs the client should use
+   * to create directories
+   * @return the ACL list
+   */
+  public List<ACL> getClientAcls() {
+    return getRegistrySecurity().getClientACLs();
+  }
+
+  /**
+   * Validate a path ... this includes checking that they are DNS-valid
+   * @param path path to validate
+   * @throws InvalidPathnameException if a path is considered invalid
+   */
+  protected void validatePath(String path) throws InvalidPathnameException {
+    RegistryPathUtils.validateElementsAsDNS(path);
+  }
+
+  @Override
+  public boolean mknode(String path, boolean createParents) throws IOException {
+    validatePath(path);
+    return zkMkPath(path, CreateMode.PERSISTENT, createParents, getClientAcls());
+  }
+
+  @Override
+  public void bind(String path,
+      ServiceRecord record,
+      int flags) throws IOException {
+    Preconditions.checkArgument(record != null, "null record");
+    validatePath(path);
+    LOG.info("Bound at {} : {}", path, record);
+
+    CreateMode mode = CreateMode.PERSISTENT;
+    byte[] bytes = serviceRecordMarshal.toByteswithHeader(record);
+    zkSet(path, mode, bytes, getClientAcls(),
+        ((flags & BindFlags.OVERWRITE) != 0));
+  }
+
+  @Override
+  public ServiceRecord resolve(String path) throws IOException {
+    byte[] bytes = zkRead(path);
+    return serviceRecordMarshal.fromBytesWithHeader(path, bytes);
+  }
+
+  @Override
+  public boolean exists(String path) throws IOException {
+    validatePath(path);
+    return zkPathExists(path);
+  }
+
+  @Override
+  public RegistryPathStatus stat(String path) throws IOException {
+    validatePath(path);
+    Stat stat = zkStat(path);
+
+    String name = RegistryPathUtils.lastPathEntry(path);
+    RegistryPathStatus status = new RegistryPathStatus(
+        name,
+        stat.getCtime(),
+        stat.getDataLength(),
+        stat.getNumChildren());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Stat {} => {}", path, status);
+    }
+    return status;
+  }
+
+  @Override
+  public List<String> list(String path) throws IOException {
+    validatePath(path);
+    return zkList(path);
+  }
+
+  @Override
+  public void delete(String path, boolean recursive) throws IOException {
+    validatePath(path);
+    zkDelete(path, recursive, null);
+  }
+
+}


[4/6] YARN-913 service registry: YARN-2652 add hadoop-yarn-registry package under hadoop-yarn

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
new file mode 100644
index 0000000..6484d28
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
@@ -0,0 +1,996 @@
+/*
+ * 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.hadoop.registry.client.impl.zk;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+import org.apache.commons.lang.StringUtils;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.util.KerberosUtil;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.service.ServiceStateException;
+import org.apache.hadoop.util.ZKUtil;
+import org.apache.zookeeper.Environment;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Locale;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+import static org.apache.hadoop.registry.client.impl.zk.ZookeeperConfigOptions.*;
+import static org.apache.hadoop.registry.client.api.RegistryConstants.*;
+
+/**
+ * Implement the registry security ... a self contained service for
+ * testability.
+ * <p>
+ * This class contains:
+ * <ol>
+ *   <li>
+ *     The registry security policy implementation, configuration reading, ACL
+ * setup and management
+ *   </li>
+ *   <li>Lots of static helper methods to aid security setup and debugging</li>
+ * </ol>
+ */
+
+public class RegistrySecurity extends AbstractService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RegistrySecurity.class);
+
+  public static final String E_UNKNOWN_AUTHENTICATION_MECHANISM =
+      "Unknown/unsupported authentication mechanism; ";
+
+  /**
+   * there's no default user to add with permissions, so it would be
+   * impossible to create nodes with unrestricted user access
+   */
+  public static final String E_NO_USER_DETERMINED_FOR_ACLS =
+      "No user for ACLs determinable from current user or registry option "
+      + KEY_REGISTRY_USER_ACCOUNTS;
+
+  /**
+   * Error raised when the registry is tagged as secure but this
+   * process doesn't have hadoop security enabled.
+   */
+  public static final String E_NO_KERBEROS =
+      "Registry security is enabled -but Hadoop security is not enabled";
+
+  /**
+   * Access policy options
+   */
+  private enum AccessPolicy {
+    anon, sasl, digest
+  }
+
+  /**
+   * Access mechanism
+   */
+  private AccessPolicy access;
+
+  /**
+   * User used for digest auth
+   */
+
+  private String digestAuthUser;
+
+  /**
+   * Password used for digest auth
+   */
+
+  private String digestAuthPassword;
+
+  /**
+   * Auth data used for digest auth
+   */
+  private byte[] digestAuthData;
+
+  /**
+   * flag set to true if the registry has security enabled.
+   */
+  private boolean secureRegistry;
+
+  /**
+   * An ACL with read-write access for anyone
+   */
+  public static final ACL ALL_READWRITE_ACCESS =
+      new ACL(ZooDefs.Perms.ALL, ZooDefs.Ids.ANYONE_ID_UNSAFE);
+
+  /**
+   * An ACL with read access for anyone
+   */
+  public static final ACL ALL_READ_ACCESS =
+      new ACL(ZooDefs.Perms.READ, ZooDefs.Ids.ANYONE_ID_UNSAFE);
+
+  /**
+   * An ACL list containing the {@link #ALL_READWRITE_ACCESS} entry.
+   * It is copy on write so can be shared without worry
+   */
+  public static final List<ACL> WorldReadWriteACL;
+
+  static {
+    List<ACL> acls = new ArrayList<ACL>();
+    acls.add(ALL_READWRITE_ACCESS);
+    WorldReadWriteACL = new CopyOnWriteArrayList<ACL>(acls);
+  }
+
+  /**
+   * the list of system ACLs
+   */
+  private final List<ACL> systemACLs = new ArrayList<ACL>();
+
+  /**
+   * A list of digest ACLs which can be added to permissions
+   * —and cleared later.
+   */
+  private final List<ACL> digestACLs = new ArrayList<ACL>();
+
+  /**
+   * the default kerberos realm
+   */
+  private String kerberosRealm;
+
+  /**
+   * Client context
+   */
+  private String jaasClientContext;
+
+  /**
+   * Client identity
+   */
+  private String jaasClientIdentity;
+
+  /**
+   * Create an instance
+   * @param name service name
+   */
+  public RegistrySecurity(String name) {
+    super(name);
+  }
+
+  /**
+   * Init the service: this sets up security based on the configuration
+   * @param conf configuration
+   * @throws Exception
+   */
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    String auth = conf.getTrimmed(KEY_REGISTRY_CLIENT_AUTH,
+        REGISTRY_CLIENT_AUTH_ANONYMOUS);
+
+    // TODO JDK7 SWITCH
+    if (REGISTRY_CLIENT_AUTH_KERBEROS.equals(auth)) {
+      access = AccessPolicy.sasl;
+    } else if (REGISTRY_CLIENT_AUTH_DIGEST.equals(auth)) {
+      access = AccessPolicy.digest;
+    } else if (REGISTRY_CLIENT_AUTH_ANONYMOUS.equals(auth)) {
+      access = AccessPolicy.anon;
+    } else {
+      throw new ServiceStateException(E_UNKNOWN_AUTHENTICATION_MECHANISM
+                                      + "\"" + auth + "\"");
+    }
+    initSecurity();
+  }
+
+  /**
+   * Init security.
+   *
+   * After this operation, the {@link #systemACLs} list is valid.
+   * @throws IOException
+   */
+  private void initSecurity() throws IOException {
+
+    secureRegistry =
+        getConfig().getBoolean(KEY_REGISTRY_SECURE, DEFAULT_REGISTRY_SECURE);
+    systemACLs.clear();
+    if (secureRegistry) {
+      addSystemACL(ALL_READ_ACCESS);
+
+      // determine the kerberos realm from JVM and settings
+      kerberosRealm = getConfig().get(KEY_REGISTRY_KERBEROS_REALM,
+          getDefaultRealmInJVM());
+
+      // System Accounts
+      String system = getOrFail(KEY_REGISTRY_SYSTEM_ACCOUNTS,
+                                DEFAULT_REGISTRY_SYSTEM_ACCOUNTS);
+
+      systemACLs.addAll(buildACLs(system, kerberosRealm, ZooDefs.Perms.ALL));
+
+      // user accounts (may be empty, but for digest one user AC must
+      // be built up
+      String user = getConfig().get(KEY_REGISTRY_USER_ACCOUNTS,
+                              DEFAULT_REGISTRY_USER_ACCOUNTS);
+      List<ACL> userACLs = buildACLs(user, kerberosRealm, ZooDefs.Perms.ALL);
+
+      // add self if the current user can be determined
+      ACL self;
+      if (UserGroupInformation.isSecurityEnabled()) {
+        self = createSaslACLFromCurrentUser(ZooDefs.Perms.ALL);
+        if (self != null) {
+          userACLs.add(self);
+        }
+      }
+
+      // here check for UGI having secure on or digest + ID
+      switch (access) {
+        case sasl:
+          // secure + SASL => has to be authenticated
+          if (!UserGroupInformation.isSecurityEnabled()) {
+            throw new IOException("Kerberos required for secure registry access");
+          }
+          UserGroupInformation currentUser =
+              UserGroupInformation.getCurrentUser();
+          jaasClientContext = getOrFail(KEY_REGISTRY_CLIENT_JAAS_CONTEXT,
+              DEFAULT_REGISTRY_CLIENT_JAAS_CONTEXT);
+          jaasClientIdentity = currentUser.getShortUserName();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Auth is SASL user=\"{}\" JAAS context=\"{}\"",
+                jaasClientIdentity,
+                jaasClientContext);
+          }
+          break;
+
+        case digest:
+          String id = getOrFail(KEY_REGISTRY_CLIENT_AUTHENTICATION_ID, "");
+          String pass = getOrFail(KEY_REGISTRY_CLIENT_AUTHENTICATION_PASSWORD, "");
+          if (userACLs.isEmpty()) {
+            //
+            throw new ServiceStateException(E_NO_USER_DETERMINED_FOR_ACLS);
+          }
+          digest(id, pass);
+          ACL acl = new ACL(ZooDefs.Perms.ALL, toDigestId(id, pass));
+          userACLs.add(acl);
+          digestAuthUser = id;
+          digestAuthPassword = pass;
+          String authPair = id + ":" + pass;
+          digestAuthData = authPair.getBytes("UTF-8");
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Auth is Digest ACL: {}", aclToString(acl));
+          }
+          break;
+
+        case anon:
+          // nothing is needed; account is read only.
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Auth is anonymous");
+          }
+          userACLs = new ArrayList<ACL>(0);
+          break;
+      }
+      systemACLs.addAll(userACLs);
+
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Registry has no security");
+      }
+      // wide open cluster, adding system acls
+      systemACLs.addAll(WorldReadWriteACL);
+    }
+  }
+
+  /**
+   * Add another system ACL
+   * @param acl add ACL
+   */
+  public void addSystemACL(ACL acl) {
+    systemACLs.add(acl);
+  }
+
+  /**
+   * Add a digest ACL
+   * @param acl add ACL
+   */
+  public boolean addDigestACL(ACL acl) {
+    if (secureRegistry) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Added ACL {}", aclToString(acl));
+      }
+      digestACLs.add(acl);
+      return true;
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Ignoring added ACL - registry is insecure{}",
+            aclToString(acl));
+      }
+      return false;
+    }
+  }
+
+  /**
+   * Reset the digest ACL list
+   */
+  public void resetDigestACLs() {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Cleared digest ACLs");
+    }
+    digestACLs.clear();
+  }
+
+  /**
+   * Flag to indicate the cluster is secure
+   * @return true if the config enabled security
+   */
+  public boolean isSecureRegistry() {
+    return secureRegistry;
+  }
+
+  /**
+   * Get the system principals
+   * @return the system principals
+   */
+  public List<ACL> getSystemACLs() {
+    Preconditions.checkNotNull(systemACLs, "registry security is unitialized");
+    return Collections.unmodifiableList(systemACLs);
+  }
+
+  /**
+   * Get all ACLs needed for a client to use when writing to the repo.
+   * That is: system ACLs, its own ACL, any digest ACLs
+   * @return the client ACLs
+   */
+  public List<ACL> getClientACLs() {
+    List<ACL> clientACLs = new ArrayList<ACL>(systemACLs);
+    clientACLs.addAll(digestACLs);
+    return clientACLs;
+  }
+
+  /**
+   * Create a SASL ACL for the user
+   * @param perms permissions
+   * @return an ACL for the current user or null if they aren't a kerberos user
+   * @throws IOException
+   */
+  public ACL createSaslACLFromCurrentUser(int perms) throws IOException {
+    UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+    if (currentUser.hasKerberosCredentials()) {
+      return createSaslACL(currentUser, perms);
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Given a UGI, create a SASL ACL from it
+   * @param ugi UGI
+   * @param perms permissions
+   * @return a new ACL
+   */
+  public ACL createSaslACL(UserGroupInformation ugi, int perms) {
+    String userName = ugi.getUserName();
+    return new ACL(perms, new Id(SCHEME_SASL, userName));
+  }
+
+  /**
+   * Get a conf option, throw an exception if it is null/empty
+   * @param key key
+   * @param defval default value
+   * @return the value
+   * @throws IOException if missing
+   */
+  private String getOrFail(String key, String defval) throws IOException {
+    String val = getConfig().get(key, defval);
+    if (StringUtils.isEmpty(val)) {
+      throw new IOException("Missing value for configuration option " + key);
+    }
+    return val;
+  }
+
+  /**
+   * Check for an id:password tuple being valid.
+   * This test is stricter than that in {@link DigestAuthenticationProvider},
+   * which splits the string, but doesn't check the contents of each
+   * half for being non-"".
+   * @param idPasswordPair id:pass pair
+   * @return true if the pass is considered valid.
+   */
+  public boolean isValid(String idPasswordPair) {
+    String[] parts = idPasswordPair.split(":");
+    return parts.length == 2
+           && !StringUtils.isEmpty(parts[0])
+           && !StringUtils.isEmpty(parts[1]);
+  }
+
+  /**
+   * Get the derived kerberos realm.
+   * @return this is built from the JVM realm, or the configuration if it
+   * overrides it. If "", it means "don't know".
+   */
+  public String getKerberosRealm() {
+    return kerberosRealm;
+  }
+
+  /**
+   * Generate a base-64 encoded digest of the idPasswordPair pair
+   * @param idPasswordPair id:password
+   * @return a string that can be used for authentication
+   */
+  public String digest(String idPasswordPair) throws IOException {
+    if (StringUtils.isEmpty(idPasswordPair) || !isValid(idPasswordPair)) {
+      throw new IOException("Invalid id:password: " + idPasswordPair);
+    }
+    try {
+      return DigestAuthenticationProvider.generateDigest(idPasswordPair);
+    } catch (NoSuchAlgorithmException e) {
+      // unlikely since it is standard to the JVM, but maybe JCE restrictions
+      // could trigger it
+      throw new IOException(e.toString(), e);
+    }
+  }
+
+  /**
+   * Generate a base-64 encoded digest of the idPasswordPair pair
+   * @param id ID
+   * @param password pass
+   * @return a string that can be used for authentication
+   * @throws IOException
+   */
+  public String digest(String id, String password) throws IOException {
+    return digest(id + ":" + password);
+  }
+
+  /**
+   * Given a digest, create an ID from it
+   * @param digest digest
+   * @return ID
+   */
+  public Id toDigestId(String digest) {
+    return new Id(SCHEME_DIGEST, digest);
+  }
+
+  /**
+   * Create a Digest ID from an id:pass pair
+   * @param id ID
+   * @param password password
+   * @return an ID
+   * @throws IOException
+   */
+  public Id toDigestId(String id, String password) throws IOException {
+    return toDigestId(digest(id, password));
+  }
+
+  /**
+   * Split up a list of the form
+   * <code>sasl:mapred@,digest:5f55d66, sasl@yarn@EXAMPLE.COM</code>
+   * into a list of possible ACL values, trimming as needed
+   *
+   * The supplied realm is added to entries where
+   * <ol>
+   *   <li>the string begins "sasl:"</li>
+   *   <li>the string ends with "@"</li>
+   * </ol>
+   * No attempt is made to validate any of the acl patterns.
+   *
+   * @param aclString list of 0 or more ACLs
+   * @param realm realm to add
+   * @return a list of split and potentially patched ACL pairs.
+   *
+   */
+  public List<String> splitAclPairs(String aclString, String realm) {
+    List<String> list = Lists.newArrayList(
+        Splitter.on(',').omitEmptyStrings().trimResults()
+                .split(aclString));
+    ListIterator<String> listIterator = list.listIterator();
+    while (listIterator.hasNext()) {
+      String next = listIterator.next();
+      if (next.startsWith(SCHEME_SASL +":") && next.endsWith("@")) {
+        listIterator.set(next + realm);
+      }
+    }
+    return list;
+  }
+
+  /**
+   * Parse a string down to an ID, adding a realm if needed
+   * @param idPair id:data tuple
+   * @param realm realm to add
+   * @return the ID.
+   * @throws IllegalArgumentException if the idPair is invalid
+   */
+  public Id parse(String idPair, String realm) {
+    int firstColon = idPair.indexOf(':');
+    int lastColon = idPair.lastIndexOf(':');
+    if (firstColon == -1 || lastColon == -1 || firstColon != lastColon) {
+      throw new IllegalArgumentException(
+          "ACL '" + idPair + "' not of expected form scheme:id");
+    }
+    String scheme = idPair.substring(0, firstColon);
+    String id = idPair.substring(firstColon + 1);
+    if (id.endsWith("@")) {
+      Preconditions.checkArgument(
+          StringUtils.isNotEmpty(realm),
+          "@ suffixed account but no realm %s", id);
+      id = id + realm;
+    }
+    return new Id(scheme, id);
+  }
+
+  /**
+   * Parse the IDs, adding a realm if needed, setting the permissions
+   * @param principalList id string
+   * @param realm realm to add
+   * @param perms permissions
+   * @return the relevant ACLs
+   * @throws IOException
+   */
+  public List<ACL> buildACLs(String principalList, String realm, int perms)
+      throws IOException {
+    List<String> aclPairs = splitAclPairs(principalList, realm);
+    List<ACL> ids = new ArrayList<ACL>(aclPairs.size());
+    for (String aclPair : aclPairs) {
+      ACL newAcl = new ACL();
+      newAcl.setId(parse(aclPair, realm));
+      newAcl.setPerms(perms);
+      ids.add(newAcl);
+    }
+    return ids;
+  }
+
+  /**
+   * Parse an ACL list. This includes configuration indirection
+   * {@link ZKUtil#resolveConfIndirection(String)}
+   * @param zkAclConf configuration string
+   * @return an ACL list
+   * @throws IOException on a bad ACL parse
+   */
+  public List<ACL> parseACLs(String zkAclConf) throws IOException {
+    try {
+      return ZKUtil.parseACLs(ZKUtil.resolveConfIndirection(zkAclConf));
+    } catch (ZKUtil.BadAclFormatException e) {
+      throw new IOException("Parsing " + zkAclConf + " :" + e, e);
+    }
+  }
+
+  /**
+   * Get the appropriate Kerberos Auth module for JAAS entries
+   * for this JVM.
+   * @return a JVM-specific kerberos login module classname.
+   */
+  public static String getKerberosAuthModuleForJVM() {
+    if (System.getProperty("java.vendor").contains("IBM")) {
+      return "com.ibm.security.auth.module.Krb5LoginModule";
+    } else {
+      return "com.sun.security.auth.module.Krb5LoginModule";
+    }
+  }
+
+  /**
+   * JAAS template: {@value}
+   * Note the semicolon on the last entry
+   */
+  private static final String JAAS_ENTRY =
+      "%s { \n"
+      + " %s required\n"
+      // kerberos module
+      + " keyTab=\"%s\"\n"
+      + " principal=\"%s\"\n"
+      + " useKeyTab=true\n"
+      + " useTicketCache=false\n"
+      + " doNotPrompt=true\n"
+      + " storeKey=true;\n"
+      + "}; \n"
+      ;
+
+  /**
+   * Create a JAAS entry for insertion
+   * @param context context of the entry
+   * @param principal kerberos principal
+   * @param keytab keytab
+   * @return a context
+   */
+  public String createJAASEntry(
+      String context,
+      String principal,
+      File keytab) {
+    Preconditions.checkArgument(StringUtils.isNotEmpty(principal),
+        "invalid principal");
+    Preconditions.checkArgument(StringUtils.isNotEmpty(context),
+        "invalid context");
+    Preconditions.checkArgument(keytab != null && keytab.isFile(),
+        "Keytab null or missing: ");
+    return String.format(
+        Locale.ENGLISH,
+        JAAS_ENTRY,
+        context,
+        getKerberosAuthModuleForJVM(),
+        keytab.getAbsolutePath(),
+        principal);
+  }
+
+  /**
+   * Bind the JVM JAS setting to the specified JAAS file.
+   *
+   * <b>Important:</b> once a file has been loaded the JVM doesn't pick up
+   * changes
+   * @param jaasFile the JAAS file
+   */
+  public static void bindJVMtoJAASFile(File jaasFile) {
+    String path = jaasFile.getAbsolutePath();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Binding {} to {}", Environment.JAAS_CONF_KEY, path);
+    }
+    System.setProperty(Environment.JAAS_CONF_KEY, path);
+  }
+
+  /**
+   * Set the Zookeeper server property
+   * {@link ZookeeperConfigOptions#PROP_ZK_SERVER_SASL_CONTEXT}
+   * to the SASL context. When the ZK server starts, this is the context
+   * which it will read in
+   * @param contextName the name of the context
+   */
+  public static void bindZKToServerJAASContext(String contextName) {
+    System.setProperty(PROP_ZK_SERVER_SASL_CONTEXT, contextName);
+  }
+
+  /**
+   * Reset any system properties related to JAAS
+   */
+  public static void clearJaasSystemProperties() {
+    System.clearProperty(Environment.JAAS_CONF_KEY);
+  }
+
+  /**
+   * Resolve the context of an entry. This is an effective test of
+   * JAAS setup, because it will relay detected problems up
+   * @param context context name
+   * @return the entry
+   * @throws RuntimeException if there is no context entry found
+   */
+  public static AppConfigurationEntry[] validateContext(String context)  {
+    if (context == null) {
+      throw new RuntimeException("Null context argument");
+    }
+    if (context.isEmpty()) {
+      throw new RuntimeException("Empty context argument");
+    }
+    javax.security.auth.login.Configuration configuration =
+        javax.security.auth.login.Configuration.getConfiguration();
+    AppConfigurationEntry[] entries =
+        configuration.getAppConfigurationEntry(context);
+    if (entries == null) {
+      throw new RuntimeException(
+          String.format("Entry \"%s\" not found; " +
+                        "JAAS config = %s",
+              context,
+              describeProperty(Environment.JAAS_CONF_KEY) ));
+    }
+    return entries;
+  }
+
+  /**
+   * Apply the security environment to this curator instance. This
+   * may include setting up the ZK system properties for SASL
+   * @param builder curator builder
+   */
+  public void applySecurityEnvironment(CuratorFrameworkFactory.Builder builder) {
+
+    if (isSecureRegistry()) {
+      switch (access) {
+        case anon:
+          clearZKSaslClientProperties();
+          break;
+
+        case digest:
+          // no SASL
+          clearZKSaslClientProperties();
+          builder.authorization(SCHEME_DIGEST, digestAuthData);
+          break;
+
+        case sasl:
+          // bind to the current identity and context within the JAAS file
+          setZKSaslClientProperties(jaasClientIdentity, jaasClientContext);
+      }
+    }
+  }
+
+  /**
+   * Set the client properties. This forces the ZK client into
+   * failing if it can't auth.
+   * <b>Important:</b>This is JVM-wide.
+   * @param username username
+   * @param context login context
+   * @throws RuntimeException if the context cannot be found in the current
+   * JAAS context
+   */
+  public static void setZKSaslClientProperties(String username,
+      String context)  {
+    RegistrySecurity.validateContext(context);
+    enableZookeeperClientSASL();
+    System.setProperty(PROP_ZK_SASL_CLIENT_USERNAME, username);
+    System.setProperty(PROP_ZK_SASL_CLIENT_CONTEXT, context);
+  }
+
+  /**
+   * Clear all the ZK SASL Client properties
+   * <b>Important:</b>This is JVM-wide
+   */
+  public static void clearZKSaslClientProperties() {
+    disableZookeeperClientSASL();
+    System.clearProperty(PROP_ZK_SASL_CLIENT_CONTEXT);
+    System.clearProperty(PROP_ZK_SASL_CLIENT_USERNAME);
+  }
+
+  /**
+   * Turn ZK SASL on
+   * <b>Important:</b>This is JVM-wide
+   */
+  protected static void enableZookeeperClientSASL() {
+    System.setProperty(PROP_ZK_ENABLE_SASL_CLIENT, "true");
+  }
+
+  /**
+   * Force disable ZK SASL bindings.
+   * <b>Important:</b>This is JVM-wide
+   */
+  public static void disableZookeeperClientSASL() {
+    System.setProperty(ZooKeeperSaslClient.ENABLE_CLIENT_SASL_KEY, "false");
+  }
+
+  /**
+   * Is the system property enabling the SASL client set?
+   * @return true if the SASL client system property is set.
+   */
+  public static boolean isClientSASLEnabled() {
+    return ZooKeeperSaslClient.isEnabled();
+  }
+
+  /**
+   * Log details about the current Hadoop user at INFO.
+   * Robust against IOEs when trying to get the current user
+   */
+  public void logCurrentHadoopUser() {
+    try {
+      UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+      LOG.info("Current user = {}",currentUser);
+      UserGroupInformation realUser = currentUser.getRealUser();
+      LOG.info("Real User = {}" , realUser);
+    } catch (IOException e) {
+      LOG.warn("Failed to get current user {}, {}", e);
+    }
+  }
+
+  /**
+   * Stringify a list of ACLs for logging. Digest ACLs have their
+   * digest values stripped for security.
+   * @param acls ACL list
+   * @return a string for logs, exceptions, ...
+   */
+  public static String aclsToString(List<ACL> acls) {
+    StringBuilder builder = new StringBuilder();
+    if (acls == null) {
+      builder.append("null ACL");
+    } else {
+      builder.append('\n');
+      for (ACL acl : acls) {
+        builder.append(aclToString(acl))
+               .append(" ");
+      }
+    }
+    return builder.toString();
+  }
+
+  /**
+   * Convert an ACL to a string, with any obfuscation needed
+   * @param acl ACL
+   * @return ACL string value
+   */
+  public static String aclToString(ACL acl) {
+    return String.format(Locale.ENGLISH,
+        "0x%02x: %s",
+        acl.getPerms(),
+        idToString(acl.getId())
+    );
+  }
+
+  /**
+   * Convert an ID to a string, stripping out all but the first few characters
+   * of any digest auth hash for security reasons
+   * @param id ID
+   * @return a string description of a Zookeeper ID
+   */
+  public static String idToString(Id id) {
+    String s;
+    if (id.getScheme().equals(SCHEME_DIGEST)) {
+      String ids = id.getId();
+      int colon = ids.indexOf(':');
+      if (colon > 0) {
+        ids = ids.substring(colon + 3);
+      }
+      s = SCHEME_DIGEST + ": " + ids;
+    } else {
+      s = id.toString();
+    }
+    return s;
+  }
+
+  /**
+   * Build up low-level security diagnostics to aid debugging
+   * @return a string to use in diagnostics
+   */
+  public String buildSecurityDiagnostics() {
+    StringBuilder builder = new StringBuilder();
+    builder.append(secureRegistry ? "secure registry; "
+                          : "insecure registry; ");
+    builder.append("Access policy: ").append(access);
+
+    builder.append(", System ACLs: ").append(aclsToString(systemACLs));
+    builder.append(UgiInfo.fromCurrentUser());
+    builder.append(" Kerberos Realm: ").append(kerberosRealm).append(" ; ");
+    builder.append(describeProperty(Environment.JAAS_CONF_KEY));
+    String sasl =
+        System.getProperty(PROP_ZK_ENABLE_SASL_CLIENT,
+            DEFAULT_ZK_ENABLE_SASL_CLIENT);
+    boolean saslEnabled = Boolean.valueOf(sasl);
+    builder.append(describeProperty(PROP_ZK_ENABLE_SASL_CLIENT,
+        DEFAULT_ZK_ENABLE_SASL_CLIENT));
+    if (saslEnabled) {
+      builder.append("JAAS Client Identity")
+             .append("=")
+             .append(jaasClientIdentity)
+             .append("; ");
+      builder.append(KEY_REGISTRY_CLIENT_JAAS_CONTEXT)
+             .append("=")
+             .append(jaasClientContext)
+             .append("; ");
+      builder.append(describeProperty(PROP_ZK_SASL_CLIENT_USERNAME));
+      builder.append(describeProperty(PROP_ZK_SASL_CLIENT_CONTEXT));
+    }
+    builder.append(describeProperty(PROP_ZK_ALLOW_FAILED_SASL_CLIENTS,
+        "(undefined but defaults to true)"));
+    builder.append(describeProperty(
+        PROP_ZK_SERVER_MAINTAIN_CONNECTION_DESPITE_SASL_FAILURE));
+    return builder.toString();
+  }
+
+  private static String describeProperty(String name) {
+    return describeProperty(name, "(undefined)");
+  }
+
+  private static String describeProperty(String name, String def) {
+    return "; " + name + "=" + System.getProperty(name, def);
+  }
+
+  /**
+   * Get the default kerberos realm —returning "" if there
+   * is no realm or other problem
+   * @return the default realm of the system if it
+   * could be determined
+   */
+  public static String getDefaultRealmInJVM() {
+    try {
+      return KerberosUtil.getDefaultRealm();
+      // JDK7
+    } catch (ClassNotFoundException ignored) {
+      // ignored
+    } catch (NoSuchMethodException ignored) {
+      // ignored
+    } catch (IllegalAccessException ignored) {
+      // ignored
+    } catch (InvocationTargetException ignored) {
+      // ignored
+    }
+    return "";
+  }
+
+  /**
+   * Create an ACL For a user.
+   * @param ugi User identity
+   * @return the ACL For the specified user. Ifthe username doesn't end
+   * in "@" then the realm is added
+   */
+  public ACL createACLForUser(UserGroupInformation ugi, int perms) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Creating ACL For ", new UgiInfo(ugi));
+    }
+    if (!secureRegistry) {
+      return ALL_READWRITE_ACCESS;
+    } else {
+      return createACLfromUsername(ugi.getUserName(), perms);
+    }
+  }
+
+  /**
+   * Given a user name (short or long), create a SASL ACL
+   * @param username user name; if it doesn't contain an "@" symbol, the
+   * service's kerberos realm is added
+   * @param perms permissions
+   * @return an ACL for the user
+   */
+  public ACL createACLfromUsername(String username, int perms) {
+    if (!username.contains("@")) {
+      username = username + "@" + kerberosRealm;
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Appending kerberos realm to make {}", username);
+      }
+    }
+    return new ACL(perms, new Id(SCHEME_SASL, username));
+  }
+
+  /**
+   * On demand string-ifier for UGI with extra details
+   */
+  public static class UgiInfo {
+
+    public static UgiInfo fromCurrentUser() {
+      try {
+        return new UgiInfo(UserGroupInformation.getCurrentUser());
+      } catch (IOException e) {
+        LOG.info("Failed to get current user {}", e, e);
+        return new UgiInfo(null);
+      }
+    }
+
+    private final UserGroupInformation ugi;
+
+    public UgiInfo(UserGroupInformation ugi) {
+      this.ugi = ugi;
+    }
+
+    @Override
+    public String toString() {
+      if (ugi==null) {
+        return "(null ugi)";
+      }
+      StringBuilder builder = new StringBuilder();
+      builder.append(ugi.getUserName()).append(": ");
+      builder.append(ugi.toString());
+      builder.append(" hasKerberosCredentials=").append(
+          ugi.hasKerberosCredentials());
+      builder.append(" isFromKeytab=").append(ugi.isFromKeytab());
+      builder.append(" kerberos is enabled in Hadoop =").append(UserGroupInformation.isSecurityEnabled());
+      return builder.toString();
+    }
+
+  }
+
+  /**
+   * on-demand stringifier for a list of ACLs
+   */
+  public static class AclListInfo {
+    public final List<ACL> acls;
+
+    public AclListInfo(List<ACL> acls) {
+      this.acls = acls;
+    }
+
+    @Override
+    public String toString() {
+      return aclsToString(acls);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZKPathDumper.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZKPathDumper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZKPathDumper.java
new file mode 100644
index 0000000..3c4a730
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZKPathDumper.java
@@ -0,0 +1,133 @@
+/*
+ * 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.hadoop.registry.client.impl.zk;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.GetChildrenBuilder;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+
+import java.util.List;
+
+/**
+ * This class dumps a registry tree to a string.
+ * It does this in the <code>toString()</code> method, so it
+ * can be used in a log statement -the operation
+ * will only take place if the method is evaluated.
+ *
+ */
+@VisibleForTesting
+public class ZKPathDumper {
+
+  public static final int INDENT = 2;
+  private final CuratorFramework curator;
+  private final String root;
+  private final boolean verbose;
+
+  /**
+   * Create a path dumper -but do not dump the path until asked
+   * @param curator curator instance
+   * @param root root
+   * @param verbose verbose flag - includes more details (such as ACLs)
+   */
+  public ZKPathDumper(CuratorFramework curator,
+      String root,
+      boolean verbose) {
+    Preconditions.checkArgument(curator != null);
+    Preconditions.checkArgument(root != null);
+    this.curator = curator;
+    this.root = root;
+    this.verbose = verbose;
+  }
+
+  /**
+   * Trigger the recursive registry dump.
+   * @return a string view of the registry
+   */
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("ZK tree for ").append(root).append('\n');
+    expand(builder, root, 1);
+    return builder.toString();
+  }
+
+  /**
+   * Recursively expand the path into the supplied string builder, increasing
+   * the indentation by {@link #INDENT} as it proceeds (depth first) down
+   * the tree
+   * @param builder string build to append to
+   * @param path path to examine
+   * @param indent current indentation
+   */
+  private void expand(StringBuilder builder,
+      String path,
+      int indent) {
+    try {
+      GetChildrenBuilder childrenBuilder = curator.getChildren();
+      List<String> children = childrenBuilder.forPath(path);
+      for (String child : children) {
+        String childPath = path + "/" + child;
+        String body;
+        Stat stat = curator.checkExists().forPath(childPath);
+        StringBuilder bodyBuilder = new StringBuilder(256);
+        bodyBuilder.append("  [")
+                          .append(stat.getDataLength())
+                          .append("]");
+        if (stat.getEphemeralOwner() > 0) {
+          bodyBuilder.append("*");
+        }
+        if (verbose) {
+          // verbose: extract ACLs
+          builder.append(" -- ");
+          List<ACL> acls =
+              curator.getACL().forPath(childPath);
+          for (ACL acl : acls) {
+            builder.append(RegistrySecurity.aclToString(acl));
+            builder.append(" ");
+          }
+        }
+        body = bodyBuilder.toString();
+        // print each child
+        append(builder, indent, ' ');
+        builder.append('/').append(child);
+        builder.append(body);
+        builder.append('\n');
+        // recurse
+        expand(builder, childPath, indent + INDENT);
+      }
+    } catch (Exception e) {
+      builder.append(e.toString()).append("\n");
+    }
+  }
+
+  /**
+   * Append the specified indentation to a builder
+   * @param builder string build to append to
+   * @param indent current indentation
+   * @param c charactor to use for indentation
+   */
+  private void append(StringBuilder builder, int indent, char c) {
+    for (int i = 0; i < indent; i++) {
+      builder.append(c);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java
new file mode 100644
index 0000000..711e27c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java
@@ -0,0 +1,119 @@
+/*
+ * 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.hadoop.registry.client.impl.zk;
+
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.apache.zookeeper.server.ZooKeeperSaslServer;
+
+/**
+ * Configuration options which are internal to Zookeeper,
+ * as well as some other ZK constants
+ * <p>
+ * Zookeeper options are passed via system properties prior to the ZK
+ * Methods/classes being invoked. This implies that:
+ * <ol>
+ *   <li>There can only be one instance of a ZK client or service class
+ *   in a single JVM —else their configuration options will conflict.</li>
+ *   <li>It is safest to set these properties immediately before
+ *   invoking ZK operations.</li>
+ * </ol>
+ *
+ */
+public interface ZookeeperConfigOptions {
+  /**
+   * Enable SASL secure clients: {@value}.
+   * This is usually set to true, with ZK set to fall back to
+   * non-SASL authentication if the SASL auth fails
+   * by the property
+   * {@link #PROP_ZK_SERVER_MAINTAIN_CONNECTION_DESPITE_SASL_FAILURE}.
+   * <p>
+   * As a result, clients will default to attempting SASL-authentication,
+   * but revert to classic authentication/anonymous access on failure.
+   */
+  String PROP_ZK_ENABLE_SASL_CLIENT =
+      ZooKeeperSaslClient.ENABLE_CLIENT_SASL_KEY;
+
+  /**
+   * Default flag for the ZK client: {@value}.
+   */
+  String DEFAULT_ZK_ENABLE_SASL_CLIENT = "true";
+
+  /**
+   * System property for the JAAS client context : {@value}.
+   *
+   * For SASL authentication to work, this must point to a
+   * context within the
+   *
+   * <p>
+   *   Default value is derived from
+   *   {@link ZooKeeperSaslClient#LOGIN_CONTEXT_NAME_KEY}
+   */
+  String PROP_ZK_SASL_CLIENT_CONTEXT =
+      ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY;
+
+  /**
+   * The SASL client username: {@value}.
+   * <p>
+   * Set this to the <i>short</i> name of the client, e.g, "user",
+   * not <code>user/host</code>, or <code>user/host@REALM</code>
+   */
+  String PROP_ZK_SASL_CLIENT_USERNAME = "zookeeper.sasl.client.username";
+
+  /**
+   * The SASL Server context, referring to a context in the JVM's
+   * JAAS context file: {@value}
+   * <p>
+   */
+  String PROP_ZK_SERVER_SASL_CONTEXT =
+      ZooKeeperSaslServer.LOGIN_CONTEXT_NAME_KEY;
+
+  /**
+   * Should ZK Server allow failed SASL clients to downgrade to classic
+   * authentication on a SASL auth failure: {@value}.
+   */
+  String PROP_ZK_SERVER_MAINTAIN_CONNECTION_DESPITE_SASL_FAILURE =
+      "zookeeper.maintain_connection_despite_sasl_failure";
+
+  /**
+   * should the ZK Server Allow failed SASL clients: {@value}.
+   */
+  String PROP_ZK_ALLOW_FAILED_SASL_CLIENTS =
+      "zookeeper.allowSaslFailedClients";
+
+  /**
+   * Kerberos realm of the server: {@value}.
+   */
+  String PROP_ZK_SERVER_REALM = "zookeeper.server.realm";
+
+  /**
+   * Path to a kinit binary: {@value}.
+   * Defaults to <code>"/usr/bin/kinit"</code>
+   */
+  String PROP_ZK_KINIT_PATH = "zookeeper.kinit";
+
+  /**
+   * ID scheme for SASL: {@value}.
+   */
+  String SCHEME_SASL = "sasl";
+
+  /**
+   * ID scheme for digest auth: {@value}.
+   */
+  String SCHEME_DIGEST = "digest";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/package-info.java
new file mode 100644
index 0000000..f7ae983
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/package-info.java
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+/**
+ * Core Zookeeper support.
+ * <p>
+ * This package contains the low-level bindings to Curator and Zookeeper,
+ * including everything related to registry security.
+ * <p>
+ * The class {@link org.apache.hadoop.registry.client.impl.zk.CuratorService}
+ * is a YARN service which offers access to a Zookeeper instance via
+ * Apache Curator.
+ * <p>
+ * The {@link org.apache.hadoop.registry.client.impl.zk.RegistrySecurity}
+ * implements the security support in the registry, though a set of
+ * static methods and as a YARN service.
+ * <p>
+ * To work with ZK, system properties need to be set before invoking
+ * some operations/instantiating some objects. The definitions of these
+ * are kept in {@link org.apache.hadoop.registry.client.impl.zk.ZookeeperConfigOptions}.
+ *
+ *
+ */
+package org.apache.hadoop.registry.client.impl.zk;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/AddressTypes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/AddressTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/AddressTypes.java
new file mode 100644
index 0000000..192819c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/AddressTypes.java
@@ -0,0 +1,92 @@
+/*
+ * 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.hadoop.registry.client.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Enum of address types -as integers.
+ * Why integers and not enums? Cross platform serialization as JSON
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface AddressTypes {
+
+  /**
+   * hostname/FQDN and port pair: {@value}.
+   * The host/domain name and port are set as separate strings in the address
+   * list, e.g.
+   * <pre>
+   *   ["namenode.example.org", "50070"]
+   * </pre>
+   */
+  public static final String ADDRESS_HOSTNAME_AND_PORT = "host/port";
+
+
+  /**
+   * Path <code>/a/b/c</code> style: {@value}.
+   * The entire path is encoded in a single entry
+   *
+   * <pre>
+   *   ["/users/example/dataset"]
+   * </pre>
+   */
+  public static final String ADDRESS_PATH = "path";
+
+
+
+  /**
+   * URI entries: {@value}.
+   * <pre>
+   *   ["http://example.org"]
+   * </pre>
+   */
+  public static final String ADDRESS_URI = "uri";
+
+  /**
+   * Zookeeper addresses as a triple : {@value}.
+   * <p>
+   * These are provide as a 3 element tuple of: hostname, port
+   * and optionally path (depending on the application)
+   * <p>
+   *   A single element would be
+   * <pre>
+   *   ["zk1","2181","/registry"]
+   * </pre>
+   *  An endpoint with multiple elements would list them as
+   * <pre>
+   *   [
+   *    ["zk1","2181","/registry"]
+   *    ["zk2","1600","/registry"]
+   *   ]
+   * </pre>
+   *
+   * the third element in each entry , the path, MUST be the same in each entry.
+   * A client reading the addresses of an endpoint is free to pick any
+   * of the set, so they must be the same.
+   *
+   */
+  public static final String ADDRESS_ZOOKEEPER = "zktriple";
+
+  /**
+   * Any other address: {@value}.
+   */
+  public static final String ADDRESS_OTHER = "";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
new file mode 100644
index 0000000..51418d9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
@@ -0,0 +1,190 @@
+/*
+ * 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.hadoop.registry.client.types;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Description of a single service/component endpoint.
+ * It is designed to be marshalled as JSON.
+ * <p>
+ * Every endpoint can have more than one address entry, such as
+ * a list of URLs to a replicated service, or a (hostname, port)
+ * pair. Each of these address entries is represented as a string list,
+ * as that is the only reliably marshallable form of a tuple JSON can represent.
+ *
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public final class Endpoint implements Cloneable {
+
+  /**
+   * API implemented at the end of the binding
+   */
+  public String api;
+
+  /**
+   * Type of address. The standard types are defined in
+   * {@link AddressTypes}
+   */
+  public String addressType;
+
+  /**
+   * Protocol type. Some standard types are defined in
+   * {@link ProtocolTypes}
+   */
+  public String protocolType;
+
+  /**
+   * a list of address tuples —tuples whose format depends on the address type
+   */
+  public List<List<String>> addresses;
+
+  /**
+   * Create an empty instance.
+   */
+  public Endpoint() {
+  }
+
+  /**
+   * Create an endpoint from another endpoint.
+   * This is a deep clone with a new list of addresses.
+   * @param that the endpoint to copy from
+   */
+  public Endpoint(Endpoint that) {
+    this.api = that.api;
+    this.addressType = that.addressType;
+    this.protocolType = that.protocolType;
+    this.addresses = new ArrayList<List<String>>(that.addresses.size());
+    for (List<String> address : addresses) {
+      List<String> addr2 = new ArrayList<String>(address.size());
+      Collections.copy(address, addr2);
+    }
+  }
+
+  /**
+   * Build an endpoint with a list of addresses
+   * @param api API name
+   * @param addressType address type
+   * @param protocolType protocol type
+   * @param addrs addresses
+   */
+  public Endpoint(String api,
+      String addressType,
+      String protocolType,
+      List<List<String>> addrs) {
+    this.api = api;
+    this.addressType = addressType;
+    this.protocolType = protocolType;
+    this.addresses = new ArrayList<List<String>>();
+    if (addrs != null) {
+      addresses.addAll(addrs);
+    }
+  }
+
+  /**
+   * Build an endpoint from a list of URIs; each URI
+   * is ASCII-encoded and added to the list of addresses.
+   * @param api API name
+   * @param protocolType protocol type
+   * @param uris URIs to convert to a list of tup;les
+   */
+  public Endpoint(String api,
+      String protocolType,
+      URI... uris) {
+    this.api = api;
+    this.addressType = AddressTypes.ADDRESS_URI;
+
+    this.protocolType = protocolType;
+    List<List<String>> addrs = new ArrayList<List<String>>(uris.length);
+    for (URI uri : uris) {
+      addrs.add(RegistryTypeUtils.tuple(uri.toString()));
+    }
+    this.addresses = addrs;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("Endpoint{");
+    sb.append("api='").append(api).append('\'');
+    sb.append(", addressType='").append(addressType).append('\'');
+    sb.append(", protocolType='").append(protocolType).append('\'');
+
+    sb.append(", addresses=");
+    if (addresses != null) {
+      sb.append("[ ");
+      for (List<String> address : addresses) {
+        sb.append("[ ");
+        if (address == null) {
+          sb.append("NULL entry in address list");
+        } else {
+          for (String elt : address) {
+            sb.append('"').append(elt).append("\" ");
+          }
+        }
+        sb.append("] ");
+      };
+      sb.append("] ");
+    } else {
+      sb.append("(null) ");
+    }
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Validate the record by checking for null fields and other invalid
+   * conditions
+   * @throws NullPointerException if a field is null when it
+   * MUST be set.
+   * @throws RuntimeException on invalid entries
+   */
+  public void validate() {
+    Preconditions.checkNotNull(api, "null API field");
+    Preconditions.checkNotNull(addressType, "null addressType field");
+    Preconditions.checkNotNull(protocolType, "null protocolType field");
+    Preconditions.checkNotNull(addresses, "null addresses field");
+    for (List<String> address : addresses) {
+      Preconditions.checkNotNull(address, "null element in address");
+    }
+  }
+
+  /**
+   * Shallow clone: the lists of addresses are shared
+   * @return a cloned instance
+   * @throws CloneNotSupportedException
+   */
+  @Override
+  protected Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ProtocolTypes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ProtocolTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ProtocolTypes.java
new file mode 100644
index 0000000..f225cf0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ProtocolTypes.java
@@ -0,0 +1,104 @@
+/*
+ * 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.hadoop.registry.client.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * some common protocol types
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface ProtocolTypes {
+
+  /**
+   * Addresses are URIs of Hadoop Filesystem paths: {@value}.
+   */
+  String PROTOCOL_FILESYSTEM = "hadoop/filesystem";
+
+  /**
+   * Classic Hadoop IPC : {@value}.
+   */
+  String PROTOCOL_HADOOP_IPC = "hadoop/IPC";
+
+  /**
+   * Hadoop protocol buffers IPC: {@value}.
+   */
+  String PROTOCOL_HADOOP_IPC_PROTOBUF = "hadoop/protobuf";
+
+  /**
+   * Corba IIOP: {@value}.
+   */
+  String PROTOCOL_IIOP = "IIOP";
+
+  /**
+   * REST: {@value}.
+   */
+  String PROTOCOL_REST = "REST";
+
+  /**
+   * Java RMI: {@value}.
+   */
+  String PROTOCOL_RMI = "RMI";
+
+  /**
+   * SunOS RPC, as used by NFS and similar: {@value}.
+   */
+  String PROTOCOL_SUN_RPC = "sunrpc";
+
+  /**
+   * Thrift-based protocols: {@value}.
+   */
+  String PROTOCOL_THRIFT = "thrift";
+
+  /**
+   * Custom TCP protocol: {@value}.
+   */
+  String PROTOCOL_TCP = "tcp";
+
+  /**
+   * Custom UPC-based protocol : {@value}.
+   */
+  String PROTOCOL_UDP = "udp";
+
+  /**
+   * Default value —the protocol is unknown : "{@value}"
+   */
+  String PROTOCOL_UNKNOWN = "";
+
+  /**
+   * Web page: {@value}.
+   *
+   * This protocol implies that the URLs are designed for
+   * people to view via web browsers.
+   */
+  String PROTOCOL_WEBUI = "webui";
+
+  /**
+   * Web Services: {@value}.
+   */
+  String PROTOCOL_WSAPI = "WS-*";
+
+  /**
+   * A zookeeper binding: {@value}.
+   */
+  String PROTOCOL_ZOOKEEPER_BINDING = "zookeeper";
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/RegistryPathStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/RegistryPathStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/RegistryPathStatus.java
new file mode 100644
index 0000000..59bcadc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/RegistryPathStatus.java
@@ -0,0 +1,123 @@
+/*
+ * 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.hadoop.registry.client.types;
+
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * Output of a <code>RegistryOperations.stat()</code> call
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+@JsonIgnoreProperties(ignoreUnknown = true)
+public final class RegistryPathStatus {
+
+  /**
+   * Short path in the registry to this entry
+   */
+  public final String path;
+
+  /**
+   * Timestamp
+   */
+  public final long time;
+
+  /**
+   * Entry size in bytes, as returned by the storage infrastructure.
+   * In zookeeper, even "empty" nodes have a non-zero size.
+   */
+  public final long size;
+
+  /**
+   * Number of child nodes
+   */
+  public final int children;
+
+  /**
+   * Construct an instance
+   * @param path full path
+   * @param time time
+   * @param size entry size
+   * @param children number of children
+   */
+  public RegistryPathStatus(
+      @JsonProperty("path") String path,
+      @JsonProperty("time") long time,
+      @JsonProperty("size") long size,
+      @JsonProperty("children") int children) {
+    this.path = path;
+    this.time = time;
+    this.size = size;
+    this.children = children;
+  }
+
+  /**
+   * Equality operator checks size, time and path of the entries.
+   * It does <i>not</i> check {@link #children}.
+   * @param other the other entry
+   * @return true if the entries are considered equal.
+   */
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (other == null || getClass() != other.getClass()) {
+      return false;
+    }
+
+    RegistryPathStatus status = (RegistryPathStatus) other;
+
+    if (size != status.size) {
+      return false;
+    }
+    if (time != status.time) {
+      return false;
+    }
+    if (path != null ? !path.equals(status.path) : status.path != null) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * The hash code is derived from the path.
+   * @return hash code for storing the path in maps.
+   */
+  @Override
+  public int hashCode() {
+    return path != null ? path.hashCode() : 0;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("RegistryPathStatus{");
+    sb.append("path='").append(path).append('\'');
+    sb.append(", time=").append(time);
+    sb.append(", size=").append(size);
+    sb.append(", children=").append(children);
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
new file mode 100644
index 0000000..378127f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
@@ -0,0 +1,249 @@
+/*
+ * 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.hadoop.registry.client.types;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.codehaus.jackson.annotate.JsonAnyGetter;
+import org.codehaus.jackson.annotate.JsonAnySetter;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * JSON-marshallable description of a single component.
+ * It supports the deserialization of unknown attributes, but does
+ * not support their creation.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class ServiceRecord implements Cloneable {
+
+  /**
+   * Description string
+   */
+  public String description;
+
+  /**
+   * map to handle unknown attributes.
+   */
+  private Map<String, String> attributes = new HashMap<String, String>(4);
+
+  /**
+   * List of endpoints intended for use to external callers
+   */
+  public List<Endpoint> external = new ArrayList<Endpoint>();
+
+  /**
+   * List of endpoints for use <i>within</i> an application.
+   */
+  public List<Endpoint> internal = new ArrayList<Endpoint>();
+
+  /**
+   * Create a service record with no ID, description or registration time.
+   * Endpoint lists are set to empty lists.
+   */
+  public ServiceRecord() {
+  }
+
+  /**
+   * Deep cloning constructor
+   * @param that service record source
+   */
+  public ServiceRecord(ServiceRecord that) {
+    this.description = that.description;
+    // others
+    Map<String, String> thatAttrs = that.attributes;
+    for (Map.Entry<String, String> entry : thatAttrs.entrySet()) {
+      attributes.put(entry.getKey(), entry.getValue());
+    }
+    // endpoints
+    List<Endpoint> src = that.internal;
+    if (src != null) {
+      internal = new ArrayList<Endpoint>(src.size());
+      for (Endpoint endpoint : src) {
+        internal.add(new Endpoint(endpoint));
+      }
+    }
+    src = that.external;
+    if (src != null) {
+      external = new ArrayList<Endpoint>(src.size());
+      for (Endpoint endpoint : src) {
+        external.add(new Endpoint(endpoint));
+      }
+    }
+  }
+
+  /**
+   * Add an external endpoint
+   * @param endpoint endpoint to set
+   */
+  public void addExternalEndpoint(Endpoint endpoint) {
+    Preconditions.checkArgument(endpoint != null);
+    endpoint.validate();
+    external.add(endpoint);
+  }
+
+  /**
+   * Add an internal endpoint
+   * @param endpoint endpoint to set
+   */
+  public void addInternalEndpoint(Endpoint endpoint) {
+    Preconditions.checkArgument(endpoint != null);
+    endpoint.validate();
+    internal.add(endpoint);
+  }
+
+  /**
+   * Look up an internal endpoint
+   * @param api API
+   * @return the endpoint or null if there was no match
+   */
+  public Endpoint getInternalEndpoint(String api) {
+    return findByAPI(internal, api);
+  }
+
+  /**
+   * Look up an external endpoint
+   * @param api API
+   * @return the endpoint or null if there was no match
+   */
+  public Endpoint getExternalEndpoint(String api) {
+    return findByAPI(external, api);
+  }
+
+  /**
+   * Handle unknown attributes by storing them in the
+   * {@link #attributes} map
+   * @param key attribute name
+   * @param value attribute value.
+   */
+  @JsonAnySetter
+  public void set(String key, Object value) {
+    attributes.put(key, value.toString());
+  }
+
+  /**
+   * The map of "other" attributes set when parsing. These
+   * are not included in the JSON value of this record when it
+   * is generated.
+   * @return a map of any unknown attributes in the deserialized JSON.
+   */
+  @JsonAnyGetter
+  public Map<String, String> attributes() {
+    return attributes;
+  }
+
+  /**
+   * Get the "other" attribute with a specific key
+   * @param key key to look up
+   * @return the value or null
+   */
+  public String get(String key) {
+    return attributes.get(key);
+  }
+
+  /**
+   * Get the "other" attribute with a specific key.
+   * @param key key to look up
+   * @param defVal default value
+   * @return the value as a string,
+   * or <code>defval</code> if the value was not present
+   */
+  public String get(String key, String defVal) {
+    String val = attributes.get(key);
+    return val != null ? val: defVal;
+  }
+
+  /**
+   * Find an endpoint by its API
+   * @param list list
+   * @param api api name
+   * @return the endpoint or null if there was no match
+   */
+  private Endpoint findByAPI(List<Endpoint> list,  String api) {
+    for (Endpoint endpoint : list) {
+      if (endpoint.api.equals(api)) {
+        return endpoint;
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("ServiceRecord{");
+    sb.append("description='").append(description).append('\'');
+    sb.append("; external endpoints: {");
+    for (Endpoint endpoint : external) {
+      sb.append(endpoint).append("; ");
+    }
+    sb.append("}; internal endpoints: {");
+    for (Endpoint endpoint : internal) {
+      sb.append(endpoint != null ? endpoint.toString() : "NULL ENDPOINT");
+      sb.append("; ");
+    }
+    sb.append('}');
+
+    if (!attributes.isEmpty()) {
+      sb.append(", attributes: {");
+      for (Map.Entry<String, String> attr : attributes.entrySet()) {
+        sb.append("\"").append(attr.getKey()).append("\"=\"")
+          .append(attr.getValue()).append("\" ");
+      }
+    } else {
+
+      sb.append(", attributes: {");
+    }
+    sb.append('}');
+
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Shallow clone: all endpoints will be shared across instances
+   * @return a clone of the instance
+   * @throws CloneNotSupportedException
+   */
+  @Override
+  protected Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+
+  /**
+   * Validate the record by checking for null fields and other invalid
+   * conditions
+   * @throws NullPointerException if a field is null when it
+   * MUST be set.
+   * @throws RuntimeException on invalid entries
+   */
+  public void validate() {
+    for (Endpoint endpoint : external) {
+      Preconditions.checkNotNull("null endpoint", endpoint);
+      endpoint.validate();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecordHeader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecordHeader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecordHeader.java
new file mode 100644
index 0000000..2f75dba
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecordHeader.java
@@ -0,0 +1,59 @@
+/*
+ * 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.hadoop.registry.client.types;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Service record header; access to the byte array kept private
+ * to avoid findbugs warnings of mutability
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class ServiceRecordHeader {
+  /**
+   * Header of a service record:  "jsonservicerec"
+   * By making this over 12 bytes long, we can auto-determine which entries
+   * in a listing are too short to contain a record without getting their data
+   */
+  private static final byte[] RECORD_HEADER = {
+      'j', 's', 'o', 'n',
+      's', 'e', 'r', 'v', 'i', 'c', 'e',
+      'r', 'e', 'c'
+  };
+
+  /**
+   * Get the length of the record header
+   * @return the header length
+   */
+  public static int getLength() {
+    return RECORD_HEADER.length;
+  }
+
+  /**
+   * Get a clone of the record header
+   * @return the new record header.
+   */
+  public static byte[] getData() {
+    byte[] h = new byte[RECORD_HEADER.length];
+    System.arraycopy(RECORD_HEADER, 0, h, 0, RECORD_HEADER.length);
+    return h;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/package-info.java
new file mode 100644
index 0000000..1c926be
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/package-info.java
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains all the data types which can be saved to the registry
+ * and/or marshalled to and from JSON.
+ * <p>
+ * The core datatypes, {@link org.apache.hadoop.registry.client.types.ServiceRecord},
+ * and {@link org.apache.hadoop.registry.client.types.Endpoint} are
+ * what is used to describe services and their protocol endpoints in the registry.
+ * <p>
+ * Some adjacent interfaces exist to list attributes of the fields:
+ * <ul>
+ *   <li>{@link org.apache.hadoop.registry.client.types.AddressTypes}</li>
+ *   <li>{@link org.apache.hadoop.registry.client.types.yarn.PersistencePolicies}</li>
+ *   <li>{@link org.apache.hadoop.registry.client.types.ProtocolTypes}</li>
+ * </ul>
+ *
+ * The {@link org.apache.hadoop.registry.client.types.RegistryPathStatus}
+ * class is not saved to the registry —it is the status of a registry
+ * entry that can be retrieved from the API call. It is still
+ * designed to be marshalled to and from JSON, as it can be served up
+ * from REST front ends to the registry.
+ *
+ */
+package org.apache.hadoop.registry.client.types;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/yarn/PersistencePolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/yarn/PersistencePolicies.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/yarn/PersistencePolicies.java
new file mode 100644
index 0000000..e4c7272
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/yarn/PersistencePolicies.java
@@ -0,0 +1,50 @@
+/*
+ * 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.hadoop.registry.client.types.yarn;
+
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+
+/**
+ * Persistence policies for {@link ServiceRecord}
+ */
+
+public interface PersistencePolicies {
+
+  /**
+   * The record persists until removed manually: {@value}.
+   */
+  String PERMANENT = "permanent";
+
+  /**
+   * Remove when the YARN application defined in the id field
+   * terminates: {@value}.
+   */
+  String APPLICATION = "application";
+
+  /**
+   * Remove when the current YARN application attempt ID finishes: {@value}.
+   */
+  String APPLICATION_ATTEMPT = "application-attempt";
+
+  /**
+   * Remove when the YARN container in the ID field finishes: {@value}
+   */
+  String CONTAINER = "container";
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/yarn/YarnRegistryAttributes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/yarn/YarnRegistryAttributes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/yarn/YarnRegistryAttributes.java
new file mode 100644
index 0000000..7b78932
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/yarn/YarnRegistryAttributes.java
@@ -0,0 +1,31 @@
+/*
+ * 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.hadoop.registry.client.types.yarn;
+
+/**
+ * YARN specific attributes in the registry
+ */
+public class YarnRegistryAttributes {
+
+  /**
+   * ID. For containers: container ID. For application instances, application ID.
+   */
+  public static final String YARN_ID = "yarn:id";
+  public static final String YARN_PERSISTENCE = "yarn:persistence";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/RMRegistryOperationsService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/RMRegistryOperationsService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/RMRegistryOperationsService.java
new file mode 100644
index 0000000..e11890f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/RMRegistryOperationsService.java
@@ -0,0 +1,246 @@
+/*
+ * 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.hadoop.registry.server.integration;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.registry.client.impl.zk.RegistryBindingSource;
+import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
+import org.apache.hadoop.registry.server.services.DeleteCompletionCallback;
+import org.apache.hadoop.registry.server.services.RegistryAdminService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.Future;
+
+/**
+ * Handle RM events by updating the registry
+ * <p>
+ * These actions are all implemented as event handlers to operations
+ * which come from the RM.
+ * <p>
+ * This service is expected to be executed by a user with the permissions
+ * to manipulate the entire registry,
+ */
+@InterfaceAudience.LimitedPrivate("YARN")
+@InterfaceStability.Evolving
+public class RMRegistryOperationsService extends RegistryAdminService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RMRegistryOperationsService.class);
+
+  private PurgePolicy purgeOnCompletionPolicy = PurgePolicy.PurgeAll;
+
+  public RMRegistryOperationsService(String name) {
+    this(name, null);
+  }
+
+  public RMRegistryOperationsService(String name,
+      RegistryBindingSource bindingSource) {
+    super(name, bindingSource);
+  }
+
+
+  /**
+   * Extend the parent service initialization by verifying that the
+   * service knows —in a secure cluster— the realm in which it is executing.
+   * It needs this to properly build up the user names and hence their
+   * access rights.
+   *
+   * @param conf configuration of the service
+   * @throws Exception
+   */
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+
+    verifyRealmValidity();
+  }
+
+  public PurgePolicy getPurgeOnCompletionPolicy() {
+    return purgeOnCompletionPolicy;
+  }
+
+  public void setPurgeOnCompletionPolicy(PurgePolicy purgeOnCompletionPolicy) {
+    this.purgeOnCompletionPolicy = purgeOnCompletionPolicy;
+  }
+
+  public void onApplicationAttemptRegistered(ApplicationAttemptId attemptId,
+      String host, int rpcport, String trackingurl) throws IOException {
+
+  }
+
+  public void onApplicationLaunched(ApplicationId id) throws IOException {
+
+  }
+
+  /**
+   * Actions to take as an AM registers itself with the RM.
+   * @param attemptId attempt ID
+   * @throws IOException problems
+   */
+  public void onApplicationMasterRegistered(ApplicationAttemptId attemptId) throws
+      IOException {
+  }
+
+  /**
+   * Actions to take when the AM container is completed
+   * @param containerId  container ID
+   * @throws IOException problems
+   */
+  public void onAMContainerFinished(ContainerId containerId) throws
+      IOException {
+    LOG.info("AM Container {} finished, purging application attempt records",
+        containerId);
+
+    // remove all application attempt entries
+    purgeAppAttemptRecords(containerId.getApplicationAttemptId());
+
+    // also treat as a container finish to remove container
+    // level records for the AM container
+    onContainerFinished(containerId);
+  }
+
+  /**
+   * remove all application attempt entries
+   * @param attemptId attempt ID
+   */
+  protected void purgeAppAttemptRecords(ApplicationAttemptId attemptId) {
+    purgeRecordsAsync("/",
+        attemptId.toString(),
+        PersistencePolicies.APPLICATION_ATTEMPT);
+  }
+
+  /**
+   * Actions to take when an application attempt is completed
+   * @param attemptId  application  ID
+   * @throws IOException problems
+   */
+  public void onApplicationAttemptUnregistered(ApplicationAttemptId attemptId)
+      throws IOException {
+    LOG.info("Application attempt {} unregistered, purging app attempt records",
+        attemptId);
+    purgeAppAttemptRecords(attemptId);
+  }
+
+  /**
+   * Actions to take when an application is completed
+   * @param id  application  ID
+   * @throws IOException problems
+   */
+  public void onApplicationCompleted(ApplicationId id)
+      throws IOException {
+    LOG.info("Application {} completed, purging application-level records",
+        id);
+    purgeRecordsAsync("/",
+        id.toString(),
+        PersistencePolicies.APPLICATION);
+  }
+
+  public void onApplicationAttemptAdded(ApplicationAttemptId appAttemptId) {
+  }
+
+  /**
+   * This is the event where the user is known, so the user directory
+   * can be created
+   * @param applicationId application  ID
+   * @param user username
+   * @throws IOException problems
+   */
+  public void onStateStoreEvent(ApplicationId applicationId, String user) throws
+      IOException {
+    initUserRegistryAsync(user);
+  }
+
+  /**
+   * Actions to take when the AM container is completed
+   * @param id  container ID
+   * @throws IOException problems
+   */
+  public void onContainerFinished(ContainerId id) throws IOException {
+    LOG.info("Container {} finished, purging container-level records",
+        id);
+    purgeRecordsAsync("/",
+        id.toString(),
+        PersistencePolicies.CONTAINER);
+  }
+
+  /**
+   * Queue an async operation to purge all matching records under a base path.
+   * <ol>
+   *   <li>Uses a depth first search</li>
+   *   <li>A match is on ID and persistence policy, or, if policy==-1, any match</li>
+   *   <li>If a record matches then it is deleted without any child searches</li>
+   *   <li>Deletions will be asynchronous if a callback is provided</li>
+   * </ol>
+   * @param path base path
+   * @param id ID for service record.id
+   * @param persistencePolicyMatch ID for the persistence policy to match:
+   * no match, no delete.
+   * @return a future that returns the #of records deleted
+   */
+  @VisibleForTesting
+  public Future<Integer> purgeRecordsAsync(String path,
+      String id,
+      String persistencePolicyMatch) {
+
+    return purgeRecordsAsync(path,
+        id, persistencePolicyMatch,
+        purgeOnCompletionPolicy,
+        new DeleteCompletionCallback());
+  }
+
+  /**
+   * Queue an async operation to purge all matching records under a base path.
+   * <ol>
+   *   <li>Uses a depth first search</li>
+   *   <li>A match is on ID and persistence policy, or, if policy==-1, any match</li>
+   *   <li>If a record matches then it is deleted without any child searches</li>
+   *   <li>Deletions will be asynchronous if a callback is provided</li>
+   * </ol>
+   * @param path base path
+   * @param id ID for service record.id
+   * @param persistencePolicyMatch ID for the persistence policy to match:
+   * no match, no delete.
+   * @param purgePolicy how to react to children under the entry
+   * @param callback an optional callback
+   * @return a future that returns the #of records deleted
+   */
+  @VisibleForTesting
+  public Future<Integer> purgeRecordsAsync(String path,
+      String id,
+      String persistencePolicyMatch,
+      PurgePolicy purgePolicy,
+      BackgroundCallback callback) {
+    LOG.info(" records under {} with ID {} and policy {}: {}",
+        path, id, persistencePolicyMatch);
+    return submit(
+        new AsyncPurge(path,
+            new SelectByYarnPersistence(id, persistencePolicyMatch),
+            purgePolicy,
+            callback));
+  }
+
+}


[2/6] YARN-913 service registry: YARN-2652 add hadoop-yarn-registry package under hadoop-yarn

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestMarshalling.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestMarshalling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestMarshalling.java
new file mode 100644
index 0000000..14e3b1f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestMarshalling.java
@@ -0,0 +1,121 @@
+/*
+ * 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.hadoop.registry.client.binding;
+
+import org.apache.hadoop.registry.RegistryTestHelper;
+import org.apache.hadoop.registry.client.exceptions.NoRecordException;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.client.types.ServiceRecordHeader;
+import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+
+/**
+ * Test record marshalling
+ */
+public class TestMarshalling extends RegistryTestHelper {
+  private static final Logger
+      LOG = LoggerFactory.getLogger(TestMarshalling.class);
+
+  @Rule
+  public final Timeout testTimeout = new Timeout(10000);
+  @Rule
+  public TestName methodName = new TestName();
+  private static RegistryUtils.ServiceRecordMarshal marshal;
+
+  @BeforeClass
+  public static void setupClass() {
+    marshal = new RegistryUtils.ServiceRecordMarshal();
+  }
+
+  @Test
+  public void testRoundTrip() throws Throwable {
+    String persistence = PersistencePolicies.PERMANENT;
+    ServiceRecord record = createRecord(persistence);
+    record.set("customkey","customvalue");
+    record.set("customkey2","customvalue2");
+    LOG.info(marshal.toJson(record));
+    byte[] bytes = marshal.toBytes(record);
+    ServiceRecord r2 = marshal.fromBytes("", bytes, 0);
+    assertMatches(record, r2);
+  }
+
+  @Test
+  public void testRoundTripHeaders() throws Throwable {
+    ServiceRecord record = createRecord(PersistencePolicies.CONTAINER);
+    byte[] bytes = marshal.toByteswithHeader(record);
+    ServiceRecord r2 = marshal.fromBytesWithHeader("", bytes);
+    assertMatches(record, r2);
+
+  }
+
+  @Test(expected = NoRecordException.class)
+  public void testRoundTripBadHeaders() throws Throwable {
+    ServiceRecord record = createRecord(PersistencePolicies.APPLICATION);
+    byte[] bytes = marshal.toByteswithHeader(record);
+    bytes[1] = 0x01;
+    marshal.fromBytesWithHeader("src", bytes);
+  }
+
+  @Test(expected = NoRecordException.class)
+  public void testUnmarshallHeaderTooShort() throws Throwable {
+    marshal.fromBytesWithHeader("src", new byte[]{'a'});
+  }
+
+  @Test(expected = EOFException.class)
+  public void testUnmarshallNoBody() throws Throwable {
+    byte[] bytes = ServiceRecordHeader.getData();
+    marshal.fromBytesWithHeader("src", bytes);
+  }
+
+
+  @Test
+  public void testUnknownFieldsRoundTrip() throws Throwable {
+    ServiceRecord record =
+        createRecord(PersistencePolicies.APPLICATION_ATTEMPT);
+    record.set("key", "value");
+    record.set("intval", "2");
+    assertEquals("value", record.get("key"));
+    assertEquals("2", record.get("intval"));
+    assertNull(record.get("null"));
+    assertEquals("defval", record.get("null", "defval"));
+    byte[] bytes = marshal.toByteswithHeader(record);
+    ServiceRecord r2 = marshal.fromBytesWithHeader("", bytes);
+    assertEquals("value", r2.get("key"));
+    assertEquals("2", r2.get("intval"));
+  }
+
+  @Test
+  public void testFieldPropagationInCopy() throws Throwable {
+    ServiceRecord record =
+        createRecord(PersistencePolicies.APPLICATION_ATTEMPT);
+    record.set("key", "value");
+    record.set("intval", "2");
+    ServiceRecord that = new ServiceRecord(record);
+    assertMatches(record, that);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestRegistryOperationUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestRegistryOperationUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestRegistryOperationUtils.java
new file mode 100644
index 0000000..b86e3fe
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestRegistryOperationUtils.java
@@ -0,0 +1,47 @@
+/*
+ * 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.hadoop.registry.client.binding;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for the {@link RegistryUtils} class
+ */
+public class TestRegistryOperationUtils extends Assert {
+
+  @Test
+  public void testUsernameExtractionEnvVarOverrride() throws Throwable {
+    String whoami = RegistryUtils.getCurrentUsernameUnencoded("drwho");
+    assertEquals("drwho", whoami);
+
+  }
+
+  @Test
+  public void testUsernameExtractionCurrentuser() throws Throwable {
+    String whoami = RegistryUtils.getCurrentUsernameUnencoded("");
+    String ugiUser = UserGroupInformation.getCurrentUser().getShortUserName();
+
+    assertEquals(ugiUser, whoami);
+
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestRegistryPathUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestRegistryPathUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestRegistryPathUtils.java
new file mode 100644
index 0000000..9a24f1c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/binding/TestRegistryPathUtils.java
@@ -0,0 +1,178 @@
+/*
+ * 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.hadoop.registry.client.binding;
+
+import static org.apache.hadoop.registry.client.binding.RegistryPathUtils.*;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.registry.client.exceptions.InvalidPathnameException;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestRegistryPathUtils extends Assert {
+
+
+  public static final String EURO = "\u20AC";
+
+  @Test
+  public void testFormatAscii() throws Throwable {
+
+    String in = "hostname01101101-1";
+    assertConverted(in, in);
+  }
+
+  /*
+  * Euro symbol
+   */
+  @Test
+  public void testFormatEuroSymbol() throws Throwable {
+    assertConverted("xn--lzg", EURO);
+  }
+
+  @Test
+  public void testFormatIdempotent() throws Throwable {
+    assertConverted("xn--lzg", RegistryPathUtils.encodeForRegistry(EURO));
+  }
+
+  @Test
+  public void testFormatCyrillicSpaced() throws Throwable {
+    assertConverted("xn--pa 3-k4di", "\u0413PA\u0414 3");
+  }
+
+  protected void assertConverted(String expected, String in) {
+    String out = RegistryPathUtils.encodeForRegistry(in);
+    assertEquals("Conversion of " + in, expected, out);
+  }
+
+  @Test
+  public void testPaths() throws Throwable {
+    assertCreatedPathEquals("/", "/", "");
+    assertCreatedPathEquals("/", "", "");
+    assertCreatedPathEquals("/", "", "/");
+    assertCreatedPathEquals("/", "/", "/");
+
+    assertCreatedPathEquals("/a", "/a", "");
+    assertCreatedPathEquals("/a", "/", "a");
+    assertCreatedPathEquals("/a/b", "/a", "b");
+    assertCreatedPathEquals("/a/b", "/a/", "b");
+    assertCreatedPathEquals("/a/b", "/a", "/b");
+    assertCreatedPathEquals("/a/b", "/a", "/b/");
+    assertCreatedPathEquals("/a", "/a", "/");
+    assertCreatedPathEquals("/alice", "/", "/alice");
+    assertCreatedPathEquals("/alice", "/alice", "/");
+  }
+
+
+
+
+  @Test
+  public void testComplexPaths() throws Throwable {
+    assertCreatedPathEquals("/", "", "");
+    assertCreatedPathEquals("/yarn/registry/users/hadoop/org-apache-hadoop",
+        "/yarn/registry",
+        "users/hadoop/org-apache-hadoop/");
+  }
+
+
+  private static void assertCreatedPathEquals(String expected, String base,
+      String path) throws IOException {
+    String fullPath = createFullPath(base, path);
+    assertEquals("\"" + base + "\" + \"" + path + "\" =\"" + fullPath + "\"",
+        expected, fullPath);
+  }
+
+  @Test
+  public void testSplittingEmpty() throws Throwable {
+    assertEquals(0, split("").size());
+    assertEquals(0, split("/").size());
+    assertEquals(0, split("///").size());
+  }
+
+
+  @Test
+  public void testSplitting() throws Throwable {
+    assertEquals(1, split("/a").size());
+    assertEquals(0, split("/").size());
+    assertEquals(3, split("/a/b/c").size());
+    assertEquals(3, split("/a/b/c/").size());
+    assertEquals(3, split("a/b/c").size());
+    assertEquals(3, split("/a/b//c").size());
+    assertEquals(3, split("//a/b/c/").size());
+    List<String> split = split("//a/b/c/");
+    assertEquals("a", split.get(0));
+    assertEquals("b", split.get(1));
+    assertEquals("c", split.get(2));
+  }
+
+  @Test
+  public void testParentOf() throws Throwable {
+    assertEquals("/", parentOf("/a"));
+    assertEquals("/", parentOf("/a/"));
+    assertEquals("/a", parentOf("/a/b"));
+    assertEquals("/a/b", parentOf("/a/b/c"));
+  }
+
+  @Test
+  public void testLastPathEntry() throws Throwable {
+    assertEquals("",lastPathEntry("/"));
+    assertEquals("",lastPathEntry("//"));
+    assertEquals("c",lastPathEntry("/a/b/c"));
+    assertEquals("c",lastPathEntry("/a/b/c/"));
+  }
+
+  @Test(expected = PathNotFoundException.class)
+  public void testParentOfRoot() throws Throwable {
+    parentOf("/");
+  }
+
+  @Test
+  public void testValidPaths() throws Throwable {
+    assertValidPath("/");
+    assertValidPath("/a/b/c");
+    assertValidPath("/users/drwho/org-apache-hadoop/registry/appid-55-55");
+    assertValidPath("/a50");
+  }
+
+  @Test
+  public void testInvalidPaths() throws Throwable {
+    assertInvalidPath("/a_b");
+    assertInvalidPath("/UpperAndLowerCase");
+    assertInvalidPath("/space in string");
+// Is this valid?    assertInvalidPath("/50");
+  }
+
+
+  private void assertValidPath(String path) throws InvalidPathnameException {
+    validateZKPath(path);
+  }
+
+
+  private void assertInvalidPath(String path) throws InvalidPathnameException {
+    try {
+      validateElementsAsDNS(path);
+      fail("path considered valid: " + path);
+    } catch (InvalidPathnameException expected) {
+      // expected
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/impl/CuratorEventCatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/impl/CuratorEventCatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/impl/CuratorEventCatcher.java
new file mode 100644
index 0000000..254ab79
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/impl/CuratorEventCatcher.java
@@ -0,0 +1,68 @@
+/*
+ * 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.hadoop.registry.client.impl;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * This is a little event catcher for curator asynchronous
+ * operations.
+ */
+public class CuratorEventCatcher implements BackgroundCallback {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CuratorEventCatcher.class);
+
+  public final BlockingQueue<CuratorEvent>
+      events = new LinkedBlockingQueue<CuratorEvent>(1);
+
+  private final AtomicInteger eventCounter = new AtomicInteger(0);
+
+
+  @Override
+  public void processResult(CuratorFramework client,
+      CuratorEvent event) throws
+      Exception {
+    LOG.info("received {}", event);
+    eventCounter.incrementAndGet();
+    events.put(event);
+  }
+
+
+  public int getCount() {
+    return eventCounter.get();
+  }
+
+  /**
+   * Blocking operation to take the first event off the queue
+   * @return the first event on the queue, when it arrives
+   * @throws InterruptedException if interrupted
+   */
+  public CuratorEvent take() throws InterruptedException {
+    return events.take();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/impl/TestCuratorService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/impl/TestCuratorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/impl/TestCuratorService.java
new file mode 100644
index 0000000..3c8b1d1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/impl/TestCuratorService.java
@@ -0,0 +1,249 @@
+/*
+ * 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.hadoop.registry.client.impl;
+
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.service.ServiceOperations;
+import org.apache.hadoop.registry.AbstractZKRegistryTest;
+import org.apache.hadoop.registry.client.impl.zk.CuratorService;
+import org.apache.hadoop.registry.client.impl.zk.RegistrySecurity;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.data.ACL;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Test the curator service
+ */
+public class TestCuratorService extends AbstractZKRegistryTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestCuratorService.class);
+
+
+  protected CuratorService curatorService;
+
+  public static final String MISSING = "/missing";
+  private List<ACL> rootACL;
+
+  @Before
+  public void startCurator() throws IOException {
+    createCuratorService();
+  }
+
+  @After
+  public void stopCurator() {
+    ServiceOperations.stop(curatorService);
+  }
+
+  /**
+   * Create an instance
+   */
+  protected void createCuratorService() throws IOException {
+    curatorService = new CuratorService("curatorService");
+    curatorService.init(createRegistryConfiguration());
+    curatorService.start();
+    rootACL = RegistrySecurity.WorldReadWriteACL;
+    curatorService.maybeCreate("", CreateMode.PERSISTENT, rootACL, true);
+  }
+
+  @Test
+  public void testLs() throws Throwable {
+    curatorService.zkList("/");
+  }
+
+  @Test(expected = PathNotFoundException.class)
+  public void testLsNotFound() throws Throwable {
+    List<String> ls = curatorService.zkList(MISSING);
+  }
+
+  @Test
+  public void testExists() throws Throwable {
+    assertTrue(curatorService.zkPathExists("/"));
+  }
+
+  @Test
+  public void testExistsMissing() throws Throwable {
+    assertFalse(curatorService.zkPathExists(MISSING));
+  }
+
+  @Test
+  public void testVerifyExists() throws Throwable {
+    pathMustExist("/");
+  }
+
+  @Test(expected = PathNotFoundException.class)
+  public void testVerifyExistsMissing() throws Throwable {
+    pathMustExist("/file-not-found");
+  }
+
+  @Test
+  public void testMkdirs() throws Throwable {
+    mkPath("/p1", CreateMode.PERSISTENT);
+    pathMustExist("/p1");
+    mkPath("/p1/p2", CreateMode.EPHEMERAL);
+    pathMustExist("/p1/p2");
+  }
+
+  private void mkPath(String path, CreateMode mode) throws IOException {
+    curatorService.zkMkPath(path, mode, false,
+        RegistrySecurity.WorldReadWriteACL);
+  }
+
+  public void pathMustExist(String path) throws IOException {
+    curatorService.zkPathMustExist(path);
+  }
+
+  @Test(expected = PathNotFoundException.class)
+  public void testMkdirChild() throws Throwable {
+    mkPath("/testMkdirChild/child", CreateMode.PERSISTENT);
+  }
+
+  @Test
+  public void testMaybeCreate() throws Throwable {
+    assertTrue(curatorService.maybeCreate("/p3", CreateMode.PERSISTENT,
+        RegistrySecurity.WorldReadWriteACL, false));
+    assertFalse(curatorService.maybeCreate("/p3", CreateMode.PERSISTENT,
+        RegistrySecurity.WorldReadWriteACL, false));
+  }
+
+  @Test
+  public void testRM() throws Throwable {
+    mkPath("/rm", CreateMode.PERSISTENT);
+    curatorService.zkDelete("/rm", false, null);
+    verifyNotExists("/rm");
+    curatorService.zkDelete("/rm", false, null);
+  }
+
+  @Test
+  public void testRMNonRf() throws Throwable {
+    mkPath("/rm", CreateMode.PERSISTENT);
+    mkPath("/rm/child", CreateMode.PERSISTENT);
+    try {
+      curatorService.zkDelete("/rm", false, null);
+      fail("expected a failure");
+    } catch (PathIsNotEmptyDirectoryException expected) {
+
+    }
+  }
+
+  @Test
+  public void testRMRf() throws Throwable {
+    mkPath("/rm", CreateMode.PERSISTENT);
+    mkPath("/rm/child", CreateMode.PERSISTENT);
+    curatorService.zkDelete("/rm", true, null);
+    verifyNotExists("/rm");
+    curatorService.zkDelete("/rm", true, null);
+  }
+
+
+  @Test
+  public void testBackgroundDelete() throws Throwable {
+    mkPath("/rm", CreateMode.PERSISTENT);
+    mkPath("/rm/child", CreateMode.PERSISTENT);
+    CuratorEventCatcher events = new CuratorEventCatcher();
+    curatorService.zkDelete("/rm", true, events);
+    CuratorEvent taken = events.take();
+    LOG.info("took {}", taken);
+    assertEquals(1, events.getCount());
+  }
+
+  @Test
+  public void testCreate() throws Throwable {
+
+    curatorService.zkCreate("/testcreate",
+        CreateMode.PERSISTENT, getTestBuffer(),
+        rootACL
+    );
+    pathMustExist("/testcreate");
+  }
+
+  @Test
+  public void testCreateTwice() throws Throwable {
+    byte[] buffer = getTestBuffer();
+    curatorService.zkCreate("/testcreatetwice",
+        CreateMode.PERSISTENT, buffer,
+        rootACL);
+    try {
+      curatorService.zkCreate("/testcreatetwice",
+          CreateMode.PERSISTENT, buffer,
+          rootACL);
+      fail();
+    } catch (FileAlreadyExistsException e) {
+
+    }
+  }
+
+  @Test
+  public void testCreateUpdate() throws Throwable {
+    byte[] buffer = getTestBuffer();
+    curatorService.zkCreate("/testcreateupdate",
+        CreateMode.PERSISTENT, buffer,
+        rootACL
+    );
+    curatorService.zkUpdate("/testcreateupdate", buffer);
+  }
+
+  @Test(expected = PathNotFoundException.class)
+  public void testUpdateMissing() throws Throwable {
+    curatorService.zkUpdate("/testupdatemissing", getTestBuffer());
+  }
+
+  @Test
+  public void testUpdateDirectory() throws Throwable {
+    mkPath("/testupdatedirectory", CreateMode.PERSISTENT);
+    curatorService.zkUpdate("/testupdatedirectory", getTestBuffer());
+  }
+
+  @Test
+  public void testUpdateDirectorywithChild() throws Throwable {
+    mkPath("/testupdatedirectorywithchild", CreateMode.PERSISTENT);
+    mkPath("/testupdatedirectorywithchild/child", CreateMode.PERSISTENT);
+    curatorService.zkUpdate("/testupdatedirectorywithchild", getTestBuffer());
+  }
+
+  @Test
+  public void testUseZKServiceForBinding() throws Throwable {
+    CuratorService cs2 = new CuratorService("curator", zookeeper);
+    cs2.init(new Configuration());
+    cs2.start();
+  }
+
+  protected byte[] getTestBuffer() {
+    byte[] buffer = new byte[1];
+    buffer[0] = '0';
+    return buffer;
+  }
+
+
+  public void verifyNotExists(String path) throws IOException {
+    if (curatorService.zkPathExists(path)) {
+      fail("Path should not exist: " + path);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/impl/TestMicroZookeeperService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/impl/TestMicroZookeeperService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/impl/TestMicroZookeeperService.java
new file mode 100644
index 0000000..4dfe453
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/client/impl/TestMicroZookeeperService.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.registry.client.impl;
+
+import org.apache.hadoop.service.ServiceOperations;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.registry.server.services.MicroZookeeperService;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+
+import java.io.IOException;
+
+/**
+ * Simple tests to look at the micro ZK service itself
+ */
+public class TestMicroZookeeperService extends Assert {
+
+  private MicroZookeeperService zookeeper;
+
+  @Rule
+  public final Timeout testTimeout = new Timeout(10000);
+  @Rule
+  public TestName methodName = new TestName();
+
+  @After
+  public void destroyZKServer() throws IOException {
+
+    ServiceOperations.stop(zookeeper);
+  }
+
+  @Test
+  public void testTempDirSupport() throws Throwable {
+    YarnConfiguration conf = new YarnConfiguration();
+    zookeeper = new MicroZookeeperService("t1");
+    zookeeper.init(conf);
+    zookeeper.start();
+    zookeeper.stop();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/integration/TestRegistryRMOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/integration/TestRegistryRMOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/integration/TestRegistryRMOperations.java
new file mode 100644
index 0000000..451a69b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/integration/TestRegistryRMOperations.java
@@ -0,0 +1,369 @@
+/*
+ * 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.hadoop.registry.integration;
+
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
+import org.apache.hadoop.registry.AbstractRegistryTest;
+import org.apache.hadoop.registry.client.api.BindFlags;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.impl.zk.ZKPathDumper;
+import org.apache.hadoop.registry.client.impl.CuratorEventCatcher;
+import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
+import org.apache.hadoop.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
+import org.apache.hadoop.registry.server.services.DeleteCompletionCallback;
+import org.apache.hadoop.registry.server.services.RegistryAdminService;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.inetAddrEndpoint;
+import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.restEndpoint;
+
+public class TestRegistryRMOperations extends AbstractRegistryTest {
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(TestRegistryRMOperations.class);
+
+  /**
+   * trigger a purge operation
+   * @param path path
+   * @param id yarn ID
+   * @param policyMatch policy to match ID on
+   * @param purgePolicy policy when there are children under a match
+   * @return the number purged
+   * @throws IOException
+   */
+  public int purge(String path,
+      String id,
+      String policyMatch,
+      RegistryAdminService.PurgePolicy purgePolicy) throws
+      IOException,
+      ExecutionException,
+      InterruptedException {
+    return purge(path, id, policyMatch, purgePolicy, null);
+  }
+
+  /**
+   *
+   * trigger a purge operation
+   * @param path pathn
+   * @param id yarn ID
+   * @param policyMatch policy to match ID on
+   * @param purgePolicy policy when there are children under a match
+   * @param callback optional callback
+   * @return the number purged
+   * @throws IOException
+   */
+  public int purge(String path,
+      String id,
+      String policyMatch,
+      RegistryAdminService.PurgePolicy purgePolicy,
+      BackgroundCallback callback) throws
+      IOException,
+      ExecutionException,
+      InterruptedException {
+
+    Future<Integer> future = registry.purgeRecordsAsync(path,
+        id, policyMatch, purgePolicy, callback);
+    try {
+      return future.get();
+    } catch (ExecutionException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      } else {
+        throw e;
+      }
+    }
+  }
+
+  @Test
+  public void testPurgeEntryCuratorCallback() throws Throwable {
+
+    String path = "/users/example/hbase/hbase1/";
+    ServiceRecord written = buildExampleServiceEntry(
+        PersistencePolicies.APPLICATION_ATTEMPT);
+    written.set(YarnRegistryAttributes.YARN_ID,
+        "testAsyncPurgeEntry_attempt_001");
+
+    operations.mknode(RegistryPathUtils.parentOf(path), true);
+    operations.bind(path, written, 0);
+
+    ZKPathDumper dump = registry.dumpPath(false);
+    CuratorEventCatcher events = new CuratorEventCatcher();
+
+    LOG.info("Initial state {}", dump);
+
+    // container query
+    String id = written.get(YarnRegistryAttributes.YARN_ID, "");
+    int opcount = purge("/",
+        id,
+        PersistencePolicies.CONTAINER,
+        RegistryAdminService.PurgePolicy.PurgeAll,
+        events);
+    assertPathExists(path);
+    assertEquals(0, opcount);
+    assertEquals("Event counter", 0, events.getCount());
+
+    // now the application attempt
+    opcount = purge("/",
+        id,
+        PersistencePolicies.APPLICATION_ATTEMPT,
+        RegistryAdminService.PurgePolicy.PurgeAll,
+        events);
+
+    LOG.info("Final state {}", dump);
+
+    assertPathNotFound(path);
+    assertEquals("wrong no of delete operations in " + dump, 1, opcount);
+    // and validate the callback event
+    assertEquals("Event counter", 1, events.getCount());
+  }
+
+  @Test
+  public void testAsyncPurgeEntry() throws Throwable {
+
+    String path = "/users/example/hbase/hbase1/";
+    ServiceRecord written = buildExampleServiceEntry(
+        PersistencePolicies.APPLICATION_ATTEMPT);
+    written.set(YarnRegistryAttributes.YARN_ID,
+        "testAsyncPurgeEntry_attempt_001");
+
+    operations.mknode(RegistryPathUtils.parentOf(path), true);
+    operations.bind(path, written, 0);
+
+    ZKPathDumper dump = registry.dumpPath(false);
+
+    LOG.info("Initial state {}", dump);
+
+    DeleteCompletionCallback deletions = new DeleteCompletionCallback();
+    int opcount = purge("/",
+        written.get(YarnRegistryAttributes.YARN_ID, ""),
+        PersistencePolicies.CONTAINER,
+        RegistryAdminService.PurgePolicy.PurgeAll,
+        deletions);
+    assertPathExists(path);
+
+    dump = registry.dumpPath(false);
+
+    assertEquals("wrong no of delete operations in " + dump, 0,
+        deletions.getEventCount());
+    assertEquals("wrong no of delete operations in " + dump, 0, opcount);
+
+
+    // now app attempt
+    deletions = new DeleteCompletionCallback();
+    opcount = purge("/",
+        written.get(YarnRegistryAttributes.YARN_ID, ""),
+        PersistencePolicies.APPLICATION_ATTEMPT,
+        RegistryAdminService.PurgePolicy.PurgeAll,
+        deletions);
+
+    dump = registry.dumpPath(false);
+    LOG.info("Final state {}", dump);
+
+    assertPathNotFound(path);
+    assertEquals("wrong no of delete operations in " + dump, 1,
+        deletions.getEventCount());
+    assertEquals("wrong no of delete operations in " + dump, 1, opcount);
+    // and validate the callback event
+
+  }
+
+  @Test
+  public void testPutGetContainerPersistenceServiceEntry() throws Throwable {
+
+    String path = ENTRY_PATH;
+    ServiceRecord written = buildExampleServiceEntry(
+        PersistencePolicies.CONTAINER);
+
+    operations.mknode(RegistryPathUtils.parentOf(path), true);
+    operations.bind(path, written, BindFlags.CREATE);
+    ServiceRecord resolved = operations.resolve(path);
+    validateEntry(resolved);
+    assertMatches(written, resolved);
+  }
+
+  /**
+   * Create a complex example app
+   * @throws Throwable
+   */
+  @Test
+  public void testCreateComplexApplication() throws Throwable {
+    String appId = "application_1408631738011_0001";
+    String cid = "container_1408631738011_0001_01_";
+    String cid1 = cid + "000001";
+    String cid2 = cid + "000002";
+    String appPath = USERPATH + "tomcat";
+
+    ServiceRecord webapp = createRecord(appId,
+        PersistencePolicies.APPLICATION, "tomcat-based web application",
+        null);
+    webapp.addExternalEndpoint(restEndpoint("www",
+        new URI("http", "//loadbalancer/", null)));
+
+    ServiceRecord comp1 = createRecord(cid1, PersistencePolicies.CONTAINER,
+        null,
+        null);
+    comp1.addExternalEndpoint(restEndpoint("www",
+        new URI("http", "//rack4server3:43572", null)));
+    comp1.addInternalEndpoint(
+        inetAddrEndpoint("jmx", "JMX", "rack4server3", 43573));
+
+    // Component 2 has a container lifespan
+    ServiceRecord comp2 = createRecord(cid2, PersistencePolicies.CONTAINER,
+        null,
+        null);
+    comp2.addExternalEndpoint(restEndpoint("www",
+        new URI("http", "//rack1server28:35881", null)));
+    comp2.addInternalEndpoint(
+        inetAddrEndpoint("jmx", "JMX", "rack1server28", 35882));
+
+    operations.mknode(USERPATH, false);
+    operations.bind(appPath, webapp, BindFlags.OVERWRITE);
+    String componentsPath = appPath + RegistryConstants.SUBPATH_COMPONENTS;
+    operations.mknode(componentsPath, false);
+    String dns1 = RegistryPathUtils.encodeYarnID(cid1);
+    String dns1path = componentsPath + dns1;
+    operations.bind(dns1path, comp1, BindFlags.CREATE);
+    String dns2 = RegistryPathUtils.encodeYarnID(cid2);
+    String dns2path = componentsPath + dns2;
+    operations.bind(dns2path, comp2, BindFlags.CREATE);
+
+    ZKPathDumper pathDumper = registry.dumpPath(false);
+    LOG.info(pathDumper.toString());
+
+    logRecord("tomcat", webapp);
+    logRecord(dns1, comp1);
+    logRecord(dns2, comp2);
+
+    ServiceRecord dns1resolved = operations.resolve(dns1path);
+    assertEquals("Persistence policies on resolved entry",
+        PersistencePolicies.CONTAINER,
+        dns1resolved.get(YarnRegistryAttributes.YARN_PERSISTENCE, ""));
+
+    Map<String, RegistryPathStatus> children =
+        RegistryUtils.statChildren(operations, componentsPath);
+    assertEquals(2, children.size());
+    Collection<RegistryPathStatus>
+        componentStats = children.values();
+    Map<String, ServiceRecord> records =
+        RegistryUtils.extractServiceRecords(operations,
+            componentsPath, componentStats);
+    assertEquals(2, records.size());
+    ServiceRecord retrieved1 = records.get(dns1path);
+    logRecord(retrieved1.get(YarnRegistryAttributes.YARN_ID, ""), retrieved1);
+    assertMatches(dns1resolved, retrieved1);
+    assertEquals(PersistencePolicies.CONTAINER,
+        retrieved1.get(YarnRegistryAttributes.YARN_PERSISTENCE, ""));
+
+    // create a listing under components/
+    operations.mknode(componentsPath + "subdir", false);
+
+    // this shows up in the listing of child entries
+    Map<String, RegistryPathStatus> childrenUpdated =
+        RegistryUtils.statChildren(operations, componentsPath);
+    assertEquals(3, childrenUpdated.size());
+
+    // the non-record child this is not picked up in the record listing
+    Map<String, ServiceRecord> recordsUpdated =
+
+        RegistryUtils.extractServiceRecords(operations,
+            componentsPath,
+            childrenUpdated);
+    assertEquals(2, recordsUpdated.size());
+
+    // now do some deletions.
+
+    // synchronous delete container ID 2
+
+    // fail if the app policy is chosen
+    assertEquals(0, purge("/", cid2, PersistencePolicies.APPLICATION,
+        RegistryAdminService.PurgePolicy.FailOnChildren));
+    // succeed for container
+    assertEquals(1, purge("/", cid2, PersistencePolicies.CONTAINER,
+        RegistryAdminService.PurgePolicy.FailOnChildren));
+    assertPathNotFound(dns2path);
+    assertPathExists(dns1path);
+
+    // expect a skip on children to skip
+    assertEquals(0,
+        purge("/", appId, PersistencePolicies.APPLICATION,
+            RegistryAdminService.PurgePolicy.SkipOnChildren));
+    assertPathExists(appPath);
+    assertPathExists(dns1path);
+
+    // attempt to delete app with policy of fail on children
+    try {
+      int p = purge("/",
+          appId,
+          PersistencePolicies.APPLICATION,
+          RegistryAdminService.PurgePolicy.FailOnChildren);
+      fail("expected a failure, got a purge count of " + p);
+    } catch (PathIsNotEmptyDirectoryException expected) {
+      // expected
+    }
+    assertPathExists(appPath);
+    assertPathExists(dns1path);
+
+
+    // now trigger recursive delete
+    assertEquals(1,
+        purge("/", appId, PersistencePolicies.APPLICATION,
+            RegistryAdminService.PurgePolicy.PurgeAll));
+    assertPathNotFound(appPath);
+    assertPathNotFound(dns1path);
+
+  }
+
+  @Test
+  public void testChildDeletion() throws Throwable {
+    ServiceRecord app = createRecord("app1",
+        PersistencePolicies.APPLICATION, "app",
+        null);
+    ServiceRecord container = createRecord("container1",
+        PersistencePolicies.CONTAINER, "container",
+        null);
+
+    operations.bind("/app", app, BindFlags.OVERWRITE);
+    operations.bind("/app/container", container, BindFlags.OVERWRITE);
+
+    try {
+      int p = purge("/",
+          "app1",
+          PersistencePolicies.APPLICATION,
+          RegistryAdminService.PurgePolicy.FailOnChildren);
+      fail("expected a failure, got a purge count of " + p);
+    } catch (PathIsNotEmptyDirectoryException expected) {
+      // expected
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/integration/TestYarnPolicySelector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/integration/TestYarnPolicySelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/integration/TestYarnPolicySelector.java
new file mode 100644
index 0000000..441b3d7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/integration/TestYarnPolicySelector.java
@@ -0,0 +1,65 @@
+/*
+ * 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.hadoop.registry.integration;
+
+import org.apache.hadoop.registry.RegistryTestHelper;
+import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
+import org.apache.hadoop.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.server.integration.SelectByYarnPersistence;
+import org.apache.hadoop.registry.server.services.RegistryAdminService;
+import org.junit.Test;
+
+public class TestYarnPolicySelector extends RegistryTestHelper {
+
+
+  private ServiceRecord record = createRecord("1",
+      PersistencePolicies.APPLICATION, "one",
+      null);
+  private RegistryPathStatus status = new RegistryPathStatus("/", 0, 0, 1);
+
+  public void assertSelected(boolean outcome,
+      RegistryAdminService.NodeSelector selector) {
+    boolean select = selector.shouldSelect("/", status, record);
+    assertEquals(selector.toString(), outcome, select);
+  }
+
+  @Test
+  public void testByContainer() throws Throwable {
+    assertSelected(false,
+        new SelectByYarnPersistence("1",
+            PersistencePolicies.CONTAINER));
+  }
+
+  @Test
+  public void testByApp() throws Throwable {
+    assertSelected(true,
+        new SelectByYarnPersistence("1",
+            PersistencePolicies.APPLICATION));
+  }
+
+
+  @Test
+  public void testByAppName() throws Throwable {
+    assertSelected(false,
+        new SelectByYarnPersistence("2",
+            PersistencePolicies.APPLICATION));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/operations/TestRegistryOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/operations/TestRegistryOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/operations/TestRegistryOperations.java
new file mode 100644
index 0000000..1cfb025
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/operations/TestRegistryOperations.java
@@ -0,0 +1,304 @@
+/*
+ * 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.hadoop.registry.operations;
+
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.registry.AbstractRegistryTest;
+import org.apache.hadoop.registry.client.api.BindFlags;
+import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.exceptions.NoRecordException;
+import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
+import org.apache.hadoop.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TestRegistryOperations extends AbstractRegistryTest {
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(TestRegistryOperations.class);
+
+  @Test
+  public void testPutGetServiceEntry() throws Throwable {
+    ServiceRecord written = putExampleServiceEntry(ENTRY_PATH, 0,
+        PersistencePolicies.APPLICATION);
+    ServiceRecord resolved = operations.resolve(ENTRY_PATH);
+    validateEntry(resolved);
+    assertMatches(written, resolved);
+  }
+
+  @Test
+  public void testDeleteServiceEntry() throws Throwable {
+    putExampleServiceEntry(ENTRY_PATH, 0);
+    operations.delete(ENTRY_PATH, false);
+  }
+
+  @Test
+  public void testDeleteNonexistentEntry() throws Throwable {
+    operations.delete(ENTRY_PATH, false);
+    operations.delete(ENTRY_PATH, true);
+  }
+
+  @Test
+  public void testStat() throws Throwable {
+    putExampleServiceEntry(ENTRY_PATH, 0);
+    RegistryPathStatus stat = operations.stat(ENTRY_PATH);
+    assertTrue(stat.size > 0);
+    assertTrue(stat.time > 0);
+    assertEquals(NAME, stat.path);
+  }
+
+  @Test
+  public void testLsParent() throws Throwable {
+    ServiceRecord written = putExampleServiceEntry(ENTRY_PATH, 0);
+    RegistryPathStatus stat = operations.stat(ENTRY_PATH);
+
+    List<String> children = operations.list(PARENT_PATH);
+    assertEquals(1, children.size());
+    assertEquals(NAME, children.get(0));
+    Map<String, RegistryPathStatus> childStats =
+        RegistryUtils.statChildren(operations, PARENT_PATH);
+    assertEquals(1, childStats.size());
+    assertEquals(stat, childStats.get(NAME));
+
+    Map<String, ServiceRecord> records =
+        RegistryUtils.extractServiceRecords(operations,
+            PARENT_PATH,
+            childStats.values());
+    assertEquals(1, records.size());
+    ServiceRecord record = records.get(ENTRY_PATH);
+    assertNotNull(record);
+    record.validate();
+    assertMatches(written, record);
+
+  }
+
+  @Test
+  public void testDeleteNonEmpty() throws Throwable {
+    putExampleServiceEntry(ENTRY_PATH, 0);
+    try {
+      operations.delete(PARENT_PATH, false);
+      fail("Expected a failure");
+    } catch (PathIsNotEmptyDirectoryException expected) {
+      // expected; ignore
+    }
+    operations.delete(PARENT_PATH, true);
+  }
+
+  @Test(expected = PathNotFoundException.class)
+  public void testStatEmptyPath() throws Throwable {
+    operations.stat(ENTRY_PATH);
+  }
+
+  @Test(expected = PathNotFoundException.class)
+  public void testLsEmptyPath() throws Throwable {
+    operations.list(PARENT_PATH);
+  }
+
+  @Test(expected = PathNotFoundException.class)
+  public void testResolveEmptyPath() throws Throwable {
+    operations.resolve(ENTRY_PATH);
+  }
+
+  @Test
+  public void testMkdirNoParent() throws Throwable {
+    String path = ENTRY_PATH + "/missing";
+    try {
+      operations.mknode(path, false);
+      RegistryPathStatus stat = operations.stat(path);
+      fail("Got a status " + stat);
+    } catch (PathNotFoundException expected) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testDoubleMkdir() throws Throwable {
+    operations.mknode(USERPATH, false);
+    String path = USERPATH + "newentry";
+    assertTrue(operations.mknode(path, false));
+    operations.stat(path);
+    assertFalse(operations.mknode(path, false));
+  }
+
+  @Test
+  public void testPutNoParent() throws Throwable {
+    ServiceRecord record = new ServiceRecord();
+    record.set(YarnRegistryAttributes.YARN_ID, "testPutNoParent");
+    String path = "/path/without/parent";
+    try {
+      operations.bind(path, record, 0);
+      // didn't get a failure
+      // trouble
+      RegistryPathStatus stat = operations.stat(path);
+      fail("Got a status " + stat);
+    } catch (PathNotFoundException expected) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testPutMinimalRecord() throws Throwable {
+    String path = "/path/with/minimal";
+    operations.mknode(path, true);
+    ServiceRecord record = new ServiceRecord();
+    operations.bind(path, record, BindFlags.OVERWRITE);
+    ServiceRecord resolve = operations.resolve(path);
+    assertMatches(record, resolve);
+
+  }
+
+  @Test(expected = PathNotFoundException.class)
+  public void testPutNoParent2() throws Throwable {
+    ServiceRecord record = new ServiceRecord();
+    record.set(YarnRegistryAttributes.YARN_ID, "testPutNoParent");
+    String path = "/path/without/parent";
+    operations.bind(path, record, 0);
+  }
+
+  @Test
+  public void testStatDirectory() throws Throwable {
+    String empty = "/empty";
+    operations.mknode(empty, false);
+    operations.stat(empty);
+  }
+
+  @Test
+  public void testStatRootPath() throws Throwable {
+    operations.mknode("/", false);
+    operations.stat("/");
+    operations.list("/");
+    operations.list("/");
+  }
+
+  @Test
+  public void testStatOneLevelDown() throws Throwable {
+    operations.mknode("/subdir", true);
+    operations.stat("/subdir");
+  }
+
+  @Test
+  public void testLsRootPath() throws Throwable {
+    String empty = "/";
+    operations.mknode(empty, false);
+    operations.stat(empty);
+  }
+
+  @Test
+  public void testResolvePathThatHasNoEntry() throws Throwable {
+    String empty = "/empty2";
+    operations.mknode(empty, false);
+    try {
+      ServiceRecord record = operations.resolve(empty);
+      fail("expected an exception, got " + record);
+    } catch (NoRecordException expected) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testOverwrite() throws Throwable {
+    ServiceRecord written = putExampleServiceEntry(ENTRY_PATH, 0);
+    ServiceRecord resolved1 = operations.resolve(ENTRY_PATH);
+    resolved1.description = "resolved1";
+    try {
+      operations.bind(ENTRY_PATH, resolved1, 0);
+      fail("overwrite succeeded when it should have failed");
+    } catch (FileAlreadyExistsException expected) {
+      // expected
+    }
+
+    // verify there's no changed
+    ServiceRecord resolved2 = operations.resolve(ENTRY_PATH);
+    assertMatches(written, resolved2);
+    operations.bind(ENTRY_PATH, resolved1, BindFlags.OVERWRITE);
+    ServiceRecord resolved3 = operations.resolve(ENTRY_PATH);
+    assertMatches(resolved1, resolved3);
+  }
+
+  @Test
+  public void testPutGetContainerPersistenceServiceEntry() throws Throwable {
+
+    String path = ENTRY_PATH;
+    ServiceRecord written = buildExampleServiceEntry(
+        PersistencePolicies.CONTAINER);
+
+    operations.mknode(RegistryPathUtils.parentOf(path), true);
+    operations.bind(path, written, BindFlags.CREATE);
+    ServiceRecord resolved = operations.resolve(path);
+    validateEntry(resolved);
+    assertMatches(written, resolved);
+  }
+
+  @Test
+  public void testAddingWriteAccessIsNoOpEntry() throws Throwable {
+
+    assertFalse(operations.addWriteAccessor("id","pass"));
+    operations.clearWriteAccessors();
+  }
+
+  @Test
+  public void testListListFully() throws Throwable {
+    ServiceRecord r1 = new ServiceRecord();
+    ServiceRecord r2 = createRecord("i",
+        PersistencePolicies.PERMANENT, "r2");
+
+    String path = USERPATH + SC_HADOOP + "/listing" ;
+    operations.mknode(path, true);
+    String r1path = path + "/r1";
+    operations.bind(r1path, r1, 0);
+    String r2path = path + "/r2";
+    operations.bind(r2path, r2, 0);
+
+    RegistryPathStatus r1stat = operations.stat(r1path);
+    assertEquals("r1", r1stat.path);
+    RegistryPathStatus r2stat = operations.stat(r2path);
+    assertEquals("r2", r2stat.path);
+    assertNotEquals(r1stat, r2stat);
+
+    // listings now
+    List<String> list = operations.list(path);
+    assertEquals("Wrong no. of children", 2, list.size());
+    // there's no order here, so create one
+    Map<String, String> names = new HashMap<String, String>();
+    String entries = "";
+    for (String child : list) {
+      names.put(child, child);
+      entries += child + " ";
+    }
+    assertTrue("No 'r1' in " + entries,
+        names.containsKey("r1"));
+    assertTrue("No 'r2' in " + entries,
+        names.containsKey("r2"));
+
+    Map<String, RegistryPathStatus> stats =
+        RegistryUtils.statChildren(operations, path);
+    assertEquals("Wrong no. of children", 2, stats.size());
+    assertEquals(r1stat, stats.get("r1"));
+    assertEquals(r2stat, stats.get("r2"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/AbstractSecureRegistryTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/AbstractSecureRegistryTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/AbstractSecureRegistryTest.java
new file mode 100644
index 0000000..ca3f9c9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/AbstractSecureRegistryTest.java
@@ -0,0 +1,356 @@
+/*
+ * 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.hadoop.registry.secure;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.util.KerberosName;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.service.ServiceOperations;
+import org.apache.hadoop.registry.RegistryTestHelper;
+import org.apache.hadoop.registry.client.impl.zk.RegistrySecurity;
+import org.apache.hadoop.registry.client.impl.zk.ZookeeperConfigOptions;
+import org.apache.hadoop.registry.server.services.AddingCompositeService;
+import org.apache.hadoop.registry.server.services.MicroZookeeperService;
+import org.apache.hadoop.registry.server.services.MicroZookeeperServiceKeys;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+import java.io.File;
+import java.io.IOException;
+import java.security.Principal;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Add kerberos tests. This is based on the (JUnit3) KerberosSecurityTestcase
+ * and its test case, <code>TestMiniKdc</code>
+ */
+public class AbstractSecureRegistryTest extends RegistryTestHelper {
+  public static final String REALM = "EXAMPLE.COM";
+  public static final String ZOOKEEPER = "zookeeper";
+  public static final String ZOOKEEPER_LOCALHOST = "zookeeper/localhost";
+  public static final String ZOOKEEPER_REALM = "zookeeper@" + REALM;
+  public static final String ZOOKEEPER_CLIENT_CONTEXT = ZOOKEEPER;
+  public static final String ZOOKEEPER_SERVER_CONTEXT = "ZOOKEEPER_SERVER";
+  ;
+  public static final String ZOOKEEPER_LOCALHOST_REALM =
+      ZOOKEEPER_LOCALHOST + "@" + REALM;
+  public static final String ALICE = "alice";
+  public static final String ALICE_CLIENT_CONTEXT = "alice";
+  public static final String ALICE_LOCALHOST = "alice/localhost";
+  public static final String BOB = "bob";
+  public static final String BOB_CLIENT_CONTEXT = "bob";
+  public static final String BOB_LOCALHOST = "bob/localhost";
+
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractSecureRegistryTest.class);
+
+  public static final Configuration CONF;
+
+  static {
+    CONF = new Configuration();
+    CONF.set("hadoop.security.authentication", "kerberos");
+    CONF.setBoolean("hadoop.security.authorization", true);
+  }
+
+  private static final AddingCompositeService classTeardown =
+      new AddingCompositeService("classTeardown");
+
+  // static initializer guarantees it is always started
+  // ahead of any @BeforeClass methods
+  static {
+    classTeardown.init(CONF);
+    classTeardown.start();
+  }
+
+  public static final String SUN_SECURITY_KRB5_DEBUG =
+      "sun.security.krb5.debug";
+
+  private final AddingCompositeService teardown =
+      new AddingCompositeService("teardown");
+
+  protected static MiniKdc kdc;
+  protected static File keytab_zk;
+  protected static File keytab_bob;
+  protected static File keytab_alice;
+  protected static File kdcWorkDir;
+  protected static Properties kdcConf;
+  protected static RegistrySecurity registrySecurity;
+
+  @Rule
+  public final Timeout testTimeout = new Timeout(900000);
+
+  @Rule
+  public TestName methodName = new TestName();
+  protected MicroZookeeperService secureZK;
+  protected static File jaasFile;
+  private LoginContext zookeeperLogin;
+
+  /**
+   * All class initialization for this test class
+   * @throws Exception
+   */
+  @BeforeClass
+  public static void beforeSecureRegistryTestClass() throws Exception {
+    registrySecurity = new RegistrySecurity("registrySecurity");
+    registrySecurity.init(CONF);
+    setupKDCAndPrincipals();
+    RegistrySecurity.clearJaasSystemProperties();
+    RegistrySecurity.bindJVMtoJAASFile(jaasFile);
+    initHadoopSecurity();
+  }
+
+  @AfterClass
+  public static void afterSecureRegistryTestClass() throws
+      Exception {
+    describe(LOG, "teardown of class");
+    classTeardown.close();
+    teardownKDC();
+  }
+
+  /**
+   * give our thread a name
+   */
+  @Before
+  public void nameThread() {
+    Thread.currentThread().setName("JUnit");
+  }
+
+  /**
+   * For unknown reasons, the before-class setting of the JVM properties were
+   * not being picked up. This method addresses that by setting them
+   * before every test case
+   */
+  @Before
+  public void beforeSecureRegistryTest() {
+
+  }
+
+  @After
+  public void afterSecureRegistryTest() throws IOException {
+    describe(LOG, "teardown of instance");
+    teardown.close();
+    stopSecureZK();
+  }
+
+  protected static void addToClassTeardown(Service svc) {
+    classTeardown.addService(svc);
+  }
+
+  protected void addToTeardown(Service svc) {
+    teardown.addService(svc);
+  }
+
+
+  public static void teardownKDC() throws Exception {
+    if (kdc != null) {
+      kdc.stop();
+      kdc = null;
+    }
+  }
+
+  /**
+   * Sets up the KDC and a set of principals in the JAAS file
+   *
+   * @throws Exception
+   */
+  public static void setupKDCAndPrincipals() throws Exception {
+    // set up the KDC
+    File target = new File(System.getProperty("test.dir", "target"));
+    kdcWorkDir = new File(target, "kdc");
+    kdcWorkDir.mkdirs();
+    if (!kdcWorkDir.mkdirs()) {
+      assertTrue(kdcWorkDir.isDirectory());
+    }
+    kdcConf = MiniKdc.createConf();
+    kdcConf.setProperty(MiniKdc.DEBUG, "true");
+    kdc = new MiniKdc(kdcConf, kdcWorkDir);
+    kdc.start();
+
+    keytab_zk = createKeytab(ZOOKEEPER, "zookeeper.keytab");
+    keytab_alice = createKeytab(ALICE, "alice.keytab");
+    keytab_bob = createKeytab(BOB, "bob.keytab");
+
+    StringBuilder jaas = new StringBuilder(1024);
+    jaas.append(registrySecurity.createJAASEntry(ZOOKEEPER_CLIENT_CONTEXT,
+        ZOOKEEPER, keytab_zk));
+    jaas.append(registrySecurity.createJAASEntry(ZOOKEEPER_SERVER_CONTEXT,
+        ZOOKEEPER_LOCALHOST, keytab_zk));
+    jaas.append(registrySecurity.createJAASEntry(ALICE_CLIENT_CONTEXT,
+        ALICE_LOCALHOST , keytab_alice));
+    jaas.append(registrySecurity.createJAASEntry(BOB_CLIENT_CONTEXT,
+        BOB_LOCALHOST, keytab_bob));
+
+    jaasFile = new File(kdcWorkDir, "jaas.txt");
+    FileUtils.write(jaasFile, jaas.toString());
+    LOG.info("\n"+ jaas);
+    RegistrySecurity.bindJVMtoJAASFile(jaasFile);
+  }
+
+
+  //
+  protected static final String kerberosRule =
+      "RULE:[1:$1@$0](.*@EXAMPLE.COM)s/@.*//\nDEFAULT";
+
+  /**
+   * Init hadoop security by setting up the UGI config
+   */
+  public static void initHadoopSecurity() {
+
+    UserGroupInformation.setConfiguration(CONF);
+
+    KerberosName.setRules(kerberosRule);
+  }
+
+  /**
+   * Stop the secure ZK and log out the ZK account
+   */
+  public synchronized void stopSecureZK() {
+    ServiceOperations.stop(secureZK);
+    secureZK = null;
+    logout(zookeeperLogin);
+    zookeeperLogin = null;
+  }
+
+
+  public static MiniKdc getKdc() {
+    return kdc;
+  }
+
+  public static File getKdcWorkDir() {
+    return kdcWorkDir;
+  }
+
+  public static Properties getKdcConf() {
+    return kdcConf;
+  }
+
+  /**
+   * Create a secure instance
+   * @param name instance name
+   * @return the instance
+   * @throws Exception
+   */
+  protected static MicroZookeeperService createSecureZKInstance(String name)
+      throws Exception {
+    String context = ZOOKEEPER_SERVER_CONTEXT;
+    Configuration conf = new Configuration();
+
+    File testdir = new File(System.getProperty("test.dir", "target"));
+    File workDir = new File(testdir, name);
+    if (!workDir.mkdirs()) {
+      assertTrue(workDir.isDirectory());
+    }
+    System.setProperty(
+        ZookeeperConfigOptions.PROP_ZK_SERVER_MAINTAIN_CONNECTION_DESPITE_SASL_FAILURE,
+        "false");
+    RegistrySecurity.validateContext(context);
+    conf.set(MicroZookeeperServiceKeys.KEY_REGISTRY_ZKSERVICE_JAAS_CONTEXT,
+        context);
+    MicroZookeeperService secureZK = new MicroZookeeperService(name);
+    secureZK.init(conf);
+    LOG.info(secureZK.getDiagnostics());
+    return secureZK;
+  }
+
+  /**
+   * Create the keytabl for the given principal, includes
+   * raw principal and $principal/localhost
+   * @param principal principal short name
+   * @param filename filename of keytab
+   * @return file of keytab
+   * @throws Exception
+   */
+  public static File createKeytab(String principal,
+      String filename) throws Exception {
+    assertNotEmpty("empty principal", principal);
+    assertNotEmpty("empty host", filename);
+    assertNotNull("Null KDC", kdc);
+    File keytab = new File(kdcWorkDir, filename);
+    kdc.createPrincipal(keytab, principal, principal +"/localhost");
+    return keytab;
+  }
+
+  public static String getPrincipalAndRealm(String principal) {
+    return principal + "@" + getRealm();
+  }
+
+  protected static String getRealm() {
+    return kdc.getRealm();
+  }
+
+
+  /**
+   * Log in, defaulting to the client context
+   * @param principal principal
+   * @param context context
+   * @param keytab keytab
+   * @return the logged in context
+   * @throws LoginException failure to log in
+   */
+  protected LoginContext login(String principal,
+      String context, File keytab) throws LoginException {
+    LOG.info("Logging in as {} in context {} with keytab {}",
+        principal, context, keytab);
+    Set<Principal> principals = new HashSet<Principal>();
+    principals.add(new KerberosPrincipal(principal));
+    Subject subject = new Subject(false, principals, new HashSet<Object>(),
+        new HashSet<Object>());
+    LoginContext login;
+    login = new LoginContext(context, subject, null,
+        KerberosConfiguration.createClientConfig(principal, keytab));
+    login.login();
+    return login;
+  }
+
+
+  /**
+   * Start the secure ZK instance using the test method name as the path.
+   * As the entry is saved to the {@link #secureZK} field, it
+   * is automatically stopped after the test case.
+   * @throws Exception on any failure
+   */
+  protected synchronized void startSecureZK() throws Exception {
+    assertNull("Zookeeper is already running", secureZK);
+
+    zookeeperLogin = login(ZOOKEEPER_LOCALHOST,
+        ZOOKEEPER_SERVER_CONTEXT,
+        keytab_zk);
+    secureZK = createSecureZKInstance("test-" + methodName.getMethodName());
+    secureZK.start();
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/KerberosConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/KerberosConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/KerberosConfiguration.java
new file mode 100644
index 0000000..f511bf6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/KerberosConfiguration.java
@@ -0,0 +1,81 @@
+/*
+ * 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.hadoop.registry.secure;
+
+import org.apache.hadoop.security.authentication.util.KerberosUtil;
+
+import javax.security.auth.login.AppConfigurationEntry;
+import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
+
+class KerberosConfiguration extends javax.security.auth.login.Configuration {
+  private String principal;
+  private String keytab;
+  private boolean isInitiator;
+
+  KerberosConfiguration(String principal, File keytab,
+      boolean client) {
+    this.principal = principal;
+    this.keytab = keytab.getAbsolutePath();
+    this.isInitiator = client;
+  }
+
+  public static javax.security.auth.login.Configuration createClientConfig(
+      String principal,
+      File keytab) {
+    return new KerberosConfiguration(principal, keytab, true);
+  }
+
+  public static javax.security.auth.login.Configuration createServerConfig(
+      String principal,
+      File keytab) {
+    return new KerberosConfiguration(principal, keytab, false);
+  }
+
+  @Override
+  public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+    Map<String, String> options = new HashMap<String, String>();
+    options.put("keyTab", keytab);
+    options.put("principal", principal);
+    options.put("useKeyTab", "true");
+    options.put("storeKey", "true");
+    options.put("doNotPrompt", "true");
+    options.put("useTicketCache", "true");
+    options.put("renewTGT", "true");
+    options.put("refreshKrb5Config", "true");
+    options.put("isInitiator", Boolean.toString(isInitiator));
+    String ticketCache = System.getenv("KRB5CCNAME");
+    if (ticketCache != null) {
+      options.put("ticketCache", ticketCache);
+    }
+    options.put("debug", "true");
+
+    return new AppConfigurationEntry[]{
+        new AppConfigurationEntry(KerberosUtil.getKrb5LoginModuleName(),
+            AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+            options)
+    };
+  }
+
+  @Override
+  public String toString() {
+    return "KerberosConfiguration with principal " + principal;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestRegistrySecurityHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestRegistrySecurityHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestRegistrySecurityHelper.java
new file mode 100644
index 0000000..8d0dc6a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestRegistrySecurityHelper.java
@@ -0,0 +1,211 @@
+/*
+ * 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.hadoop.registry.secure;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.registry.client.impl.zk.RegistrySecurity;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.registry.client.api.RegistryConstants.*;
+
+/**
+ * Test for registry security operations
+ */
+public class TestRegistrySecurityHelper extends Assert {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRegistrySecurityHelper.class);
+
+  public static final String YARN_EXAMPLE_COM = "yarn@example.com";
+  public static final String SASL_YARN_EXAMPLE_COM =
+      "sasl:" + YARN_EXAMPLE_COM;
+  public static final String MAPRED_EXAMPLE_COM = "mapred@example.com";
+  public static final String SASL_MAPRED_EXAMPLE_COM =
+      "sasl:" + MAPRED_EXAMPLE_COM;
+  public static final String SASL_MAPRED_APACHE = "sasl:mapred@APACHE";
+  public static final String DIGEST_F0AF = "digest:f0afbeeb00baa";
+  public static final String SASL_YARN_SHORT = "sasl:yarn@";
+  public static final String SASL_MAPRED_SHORT = "sasl:mapred@";
+  public static final String REALM_EXAMPLE_COM = "example.com";
+  private static RegistrySecurity registrySecurity;
+
+  @BeforeClass
+  public static void setupTestRegistrySecurityHelper() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setBoolean(KEY_REGISTRY_SECURE, true);
+    conf.set(KEY_REGISTRY_KERBEROS_REALM, "KERBEROS");
+    registrySecurity = new RegistrySecurity("");
+    // init the ACLs OUTSIDE A KERBEROS CLUSTER
+    registrySecurity.init(conf);
+  }
+
+  @Test
+  public void testACLSplitRealmed() throws Throwable {
+    List<String> pairs =
+        registrySecurity.splitAclPairs(
+            SASL_YARN_EXAMPLE_COM +
+            ", " +
+            SASL_MAPRED_EXAMPLE_COM,
+            "");
+
+    assertEquals(SASL_YARN_EXAMPLE_COM, pairs.get(0));
+    assertEquals(SASL_MAPRED_EXAMPLE_COM, pairs.get(1));
+  }
+
+
+  @Test
+  public void testBuildAclsRealmed() throws Throwable {
+    List<ACL> acls = registrySecurity.buildACLs(
+        SASL_YARN_EXAMPLE_COM +
+        ", " +
+        SASL_MAPRED_EXAMPLE_COM,
+        "",
+        ZooDefs.Perms.ALL);
+    assertEquals(YARN_EXAMPLE_COM, acls.get(0).getId().getId());
+    assertEquals(MAPRED_EXAMPLE_COM, acls.get(1).getId().getId());
+  }
+
+  @Test
+  public void testACLDefaultRealm() throws Throwable {
+    List<String> pairs =
+        registrySecurity.splitAclPairs(
+            SASL_YARN_SHORT +
+            ", " +
+            SASL_MAPRED_SHORT,
+            REALM_EXAMPLE_COM);
+
+    assertEquals(SASL_YARN_EXAMPLE_COM, pairs.get(0));
+    assertEquals(SASL_MAPRED_EXAMPLE_COM, pairs.get(1));
+  }
+
+  @Test
+  public void testBuildAclsDefaultRealm() throws Throwable {
+    List<ACL> acls = registrySecurity.buildACLs(
+        SASL_YARN_SHORT +
+        ", " +
+        SASL_MAPRED_SHORT,
+        REALM_EXAMPLE_COM, ZooDefs.Perms.ALL);
+
+    assertEquals(YARN_EXAMPLE_COM, acls.get(0).getId().getId());
+    assertEquals(MAPRED_EXAMPLE_COM, acls.get(1).getId().getId());
+  }
+
+  @Test
+  public void testACLSplitNullRealm() throws Throwable {
+    List<String> pairs =
+        registrySecurity.splitAclPairs(
+            SASL_YARN_SHORT +
+            ", " +
+            SASL_MAPRED_SHORT,
+            "");
+
+    assertEquals(SASL_YARN_SHORT, pairs.get(0));
+    assertEquals(SASL_MAPRED_SHORT, pairs.get(1));
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testBuildAclsNullRealm() throws Throwable {
+    registrySecurity.buildACLs(
+        SASL_YARN_SHORT +
+        ", " +
+        SASL_MAPRED_SHORT,
+        "", ZooDefs.Perms.ALL);
+    fail("");
+
+  }
+
+  @Test
+  public void testACLDefaultRealmOnlySASL() throws Throwable {
+    List<String> pairs =
+        registrySecurity.splitAclPairs(
+            SASL_YARN_SHORT +
+            ", " +
+            DIGEST_F0AF,
+            REALM_EXAMPLE_COM);
+
+    assertEquals(SASL_YARN_EXAMPLE_COM, pairs.get(0));
+    assertEquals(DIGEST_F0AF, pairs.get(1));
+  }
+
+  @Test
+  public void testACLSplitMixed() throws Throwable {
+    List<String> pairs =
+        registrySecurity.splitAclPairs(
+            SASL_YARN_SHORT +
+            ", " +
+            SASL_MAPRED_APACHE +
+            ", ,," +
+            DIGEST_F0AF,
+            REALM_EXAMPLE_COM);
+
+    assertEquals(SASL_YARN_EXAMPLE_COM, pairs.get(0));
+    assertEquals(SASL_MAPRED_APACHE, pairs.get(1));
+    assertEquals(DIGEST_F0AF, pairs.get(2));
+  }
+
+  @Test
+  public void testDefaultAClsValid() throws Throwable {
+    registrySecurity.buildACLs(
+        RegistryConstants.DEFAULT_REGISTRY_SYSTEM_ACCOUNTS,
+        REALM_EXAMPLE_COM, ZooDefs.Perms.ALL);
+  }
+
+  @Test
+  public void testDefaultRealm() throws Throwable {
+    String realm = RegistrySecurity.getDefaultRealmInJVM();
+    LOG.info("Realm {}", realm);
+  }
+
+  @Test
+  public void testUGIProperties() throws Throwable {
+    UserGroupInformation user = UserGroupInformation.getCurrentUser();
+    ACL acl = registrySecurity.createACLForUser(user, ZooDefs.Perms.ALL);
+    assertFalse(RegistrySecurity.ALL_READWRITE_ACCESS.equals(acl));
+    LOG.info("User {} has ACL {}", user, acl);
+  }
+
+
+  @Test
+  public void testSecurityImpliesKerberos() throws Throwable {
+    Configuration conf = new Configuration();
+    conf.setBoolean("hadoop.security.authentication", true);
+    conf.setBoolean(KEY_REGISTRY_SECURE, true);
+    conf.set(KEY_REGISTRY_KERBEROS_REALM, "KERBEROS");
+    RegistrySecurity security = new RegistrySecurity("registry security");
+    try {
+      security.init(conf);
+    } catch (Exception e) {
+      assertTrue(
+          "did not find "+ RegistrySecurity.E_NO_KERBEROS + " in " + e,
+          e.toString().contains(RegistrySecurity.E_NO_KERBEROS));
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureLogins.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureLogins.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureLogins.java
new file mode 100644
index 0000000..ab9d490
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureLogins.java
@@ -0,0 +1,214 @@
+/*
+ * 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.hadoop.registry.secure;
+
+
+
+import com.sun.security.auth.module.Krb5LoginModule;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.security.HadoopKerberosName;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.util.KerberosName;
+import org.apache.hadoop.security.authentication.util.KerberosUtil;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.registry.client.impl.zk.RegistrySecurity;
+import org.apache.hadoop.registry.client.impl.zk.ZookeeperConfigOptions;
+import org.apache.zookeeper.Environment;
+import org.apache.zookeeper.data.ACL;
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+import java.io.File;
+import java.io.IOException;
+import java.security.Principal;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Verify that logins work
+ */
+public class TestSecureLogins extends AbstractSecureRegistryTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestSecureLogins.class);
+
+  @Test
+  public void testZKinKeytab() throws Throwable {
+    Assume.assumeTrue(!Shell.WINDOWS);
+    try {
+      String listing = ktList(keytab_zk);
+      assertTrue("no " + ZOOKEEPER_LOCALHOST + " in " + listing,
+          listing.contains(ZOOKEEPER_LOCALHOST));
+    } catch (IOException e) {
+      LOG.debug(KTUTIL + " failure: {}", e, e);
+      Assume.assumeTrue("Failed to run "+ KTUTIL+": " + e, false );
+    }
+  }
+
+  @Test
+  public void testHasRealm() throws Throwable {
+    assertNotNull(getRealm());
+    LOG.info("ZK principal = {}", getPrincipalAndRealm(ZOOKEEPER_LOCALHOST));
+  }
+
+  @Test
+  public void testJaasFileSetup() throws Throwable {
+    // the JVM has seemed inconsistent on setting up here
+    assertNotNull("jaasFile", jaasFile);
+    String confFilename = System.getProperty(Environment.JAAS_CONF_KEY);
+    assertEquals(jaasFile.getAbsolutePath(), confFilename);
+  }
+
+  @Test
+  public void testJaasFileBinding() throws Throwable {
+    // the JVM has seemed inconsistent on setting up here
+    assertNotNull("jaasFile", jaasFile);
+    RegistrySecurity.bindJVMtoJAASFile(jaasFile);
+    String confFilename = System.getProperty(Environment.JAAS_CONF_KEY);
+    assertEquals(jaasFile.getAbsolutePath(), confFilename);
+  }
+
+
+  @Test
+  public void testClientLogin() throws Throwable {
+    LoginContext client = login(ALICE_LOCALHOST,
+                                ALICE_CLIENT_CONTEXT,
+                                keytab_alice);
+
+    logLoginDetails(ALICE_LOCALHOST, client);
+    String confFilename = System.getProperty(Environment.JAAS_CONF_KEY);
+    assertNotNull("Unset: "+ Environment.JAAS_CONF_KEY, confFilename);
+    String config = FileUtils.readFileToString(new File(confFilename));
+    LOG.info("{}=\n{}", confFilename, config);
+    RegistrySecurity.setZKSaslClientProperties(ALICE, ALICE_CLIENT_CONTEXT);
+    client.logout();
+  }
+
+
+  @Test
+  public void testServerLogin() throws Throwable {
+    LoginContext loginContext = createLoginContextZookeeperLocalhost();
+    loginContext.login();
+    loginContext.logout();
+  }
+
+  public LoginContext createLoginContextZookeeperLocalhost() throws
+      LoginException {
+    String principalAndRealm = getPrincipalAndRealm(ZOOKEEPER_LOCALHOST);
+    Set<Principal> principals = new HashSet<Principal>();
+    principals.add(new KerberosPrincipal(ZOOKEEPER_LOCALHOST));
+    Subject subject = new Subject(false, principals, new HashSet<Object>(),
+        new HashSet<Object>());
+    return new LoginContext("", subject, null,
+        KerberosConfiguration.createServerConfig(ZOOKEEPER_LOCALHOST, keytab_zk));
+  }
+
+
+  @Test
+  public void testKerberosAuth() throws Throwable {
+    File krb5conf = getKdc().getKrb5conf();
+    String krbConfig = FileUtils.readFileToString(krb5conf);
+    LOG.info("krb5.conf at {}:\n{}", krb5conf, krbConfig);
+    Subject subject = new Subject();
+
+    final Krb5LoginModule krb5LoginModule = new Krb5LoginModule();
+    final Map<String, String> options = new HashMap<String, String>();
+    options.put("keyTab", keytab_alice.getAbsolutePath());
+    options.put("principal", ALICE_LOCALHOST);
+    options.put("debug", "true");
+    options.put("doNotPrompt", "true");
+    options.put("isInitiator", "true");
+    options.put("refreshKrb5Config", "true");
+    options.put("renewTGT", "true");
+    options.put("storeKey", "true");
+    options.put("useKeyTab", "true");
+    options.put("useTicketCache", "true");
+
+    krb5LoginModule.initialize(subject, null,
+        new HashMap<String, String>(),
+        options);
+
+    boolean loginOk = krb5LoginModule.login();
+    assertTrue("Failed to login", loginOk);
+    boolean commitOk = krb5LoginModule.commit();
+    assertTrue("Failed to Commit", commitOk);
+  }
+
+  @Test
+  public void testDefaultRealmValid() throws Throwable {
+    String defaultRealm = KerberosUtil.getDefaultRealm();
+    assertNotEmpty("No default Kerberos Realm",
+        defaultRealm);
+    LOG.info("Default Realm '{}'", defaultRealm);
+  }
+
+  @Test
+  public void testKerberosRulesValid() throws Throwable {
+    assertTrue("!KerberosName.hasRulesBeenSet()",
+        KerberosName.hasRulesBeenSet());
+    String rules = KerberosName.getRules();
+    assertEquals(kerberosRule, rules);
+    LOG.info(rules);
+  }
+
+  @Test
+  public void testValidKerberosName() throws Throwable {
+
+    new HadoopKerberosName(ZOOKEEPER).getShortName();
+    new HadoopKerberosName(ZOOKEEPER_LOCALHOST).getShortName();
+    new HadoopKerberosName(ZOOKEEPER_REALM).getShortName();
+    // standard rules don't pick this up
+    // new HadoopKerberosName(ZOOKEEPER_LOCALHOST_REALM).getShortName();
+  }
+
+
+  @Test
+  public void testUGILogin() throws Throwable {
+
+    UserGroupInformation ugi = loginUGI(ZOOKEEPER, keytab_zk);
+    RegistrySecurity.UgiInfo ugiInfo =
+        new RegistrySecurity.UgiInfo(ugi);
+    LOG.info("logged in as: {}", ugiInfo);
+    assertTrue("security is not enabled: " + ugiInfo,
+        UserGroupInformation.isSecurityEnabled());
+    assertTrue("login is keytab based: " + ugiInfo,
+        ugi.isFromKeytab());
+
+    // now we are here, build a SASL ACL
+    ACL acl = ugi.doAs(new PrivilegedExceptionAction<ACL>() {
+      @Override
+      public ACL run() throws Exception {
+        return registrySecurity.createSaslACLFromCurrentUser(0);
+      }
+    });
+    assertEquals(ZOOKEEPER_REALM, acl.getId().getId());
+    assertEquals(ZookeeperConfigOptions.SCHEME_SASL, acl.getId().getScheme());
+    registrySecurity.addSystemACL(acl);
+
+  }
+
+}


[3/6] YARN-913 service registry: YARN-2652 add hadoop-yarn-registry package under hadoop-yarn

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/SelectByYarnPersistence.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/SelectByYarnPersistence.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/SelectByYarnPersistence.java
new file mode 100644
index 0000000..004be86
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/SelectByYarnPersistence.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.registry.server.integration;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
+import org.apache.hadoop.registry.server.services.RegistryAdminService;
+
+/**
+ * Select an entry by the YARN persistence policy
+ */
+public class SelectByYarnPersistence
+    implements RegistryAdminService.NodeSelector {
+  private final String id;
+  private final String targetPolicy;
+
+  public SelectByYarnPersistence(String id, String targetPolicy) {
+    Preconditions.checkArgument(!StringUtils.isEmpty(id), "id");
+    Preconditions.checkArgument(!StringUtils.isEmpty(targetPolicy),
+        "targetPolicy");
+    this.id = id;
+    this.targetPolicy = targetPolicy;
+  }
+
+  @Override
+  public boolean shouldSelect(String path,
+      RegistryPathStatus registryPathStatus,
+      ServiceRecord serviceRecord) {
+    String policy =
+        serviceRecord.get(YarnRegistryAttributes.YARN_PERSISTENCE, "");
+    return id.equals(serviceRecord.get(YarnRegistryAttributes.YARN_ID, ""))
+           && (targetPolicy.equals(policy));
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "Select by ID %s and policy %s: {}",
+        id, targetPolicy);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/package-info.java
new file mode 100644
index 0000000..22d8bc5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains the classes which integrate with the YARN resource
+ * manager.
+ */
+package org.apache.hadoop.registry.server.integration;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/package-info.java
new file mode 100644
index 0000000..6962eb8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Server-side classes for the registry
+ * <p>
+ *   These are components intended to be deployed only on servers or in test
+ *   JVMs, rather than on client machines.
+ * <p>
+ *   Example components are: server-side ZK support, a REST service, etc.
+ */
+package org.apache.hadoop.registry.server;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/AddingCompositeService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/AddingCompositeService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/AddingCompositeService.java
new file mode 100644
index 0000000..9faede4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/AddingCompositeService.java
@@ -0,0 +1,56 @@
+/*
+ * 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.hadoop.registry.server.services;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.service.Service;
+
+/**
+ * Composite service that exports the add/remove methods.
+ * <p>
+ * This allows external classes to add services to these methods, after which
+ * they follow the same lifecyce.
+ * <p>
+ * It is essential that any service added is in a state where it can be moved
+ * on with that of the parent services. Specifically, do not add an uninited
+ * service to a parent that is already inited —as the <code>start</code>
+ * operation will then fail
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class AddingCompositeService extends CompositeService {
+
+
+  public AddingCompositeService(String name) {
+    super(name);
+  }
+
+  @Override
+  public void addService(Service service) {
+    super.addService(service);
+  }
+
+  @Override
+  public boolean removeService(Service service) {
+    return super.removeService(service);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/DeleteCompletionCallback.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/DeleteCompletionCallback.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/DeleteCompletionCallback.java
new file mode 100644
index 0000000..e160d4a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/DeleteCompletionCallback.java
@@ -0,0 +1,58 @@
+/*
+ * 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.hadoop.registry.server.services;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.hadoop.registry.server.integration.RMRegistryOperationsService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Curator callback for delete operations completing.
+ * <p>
+ * This callback logs at debug and increments the event counter.
+ */
+public class DeleteCompletionCallback implements BackgroundCallback {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RMRegistryOperationsService.class);
+
+  private AtomicInteger events = new AtomicInteger(0);
+
+  @Override
+  public void processResult(CuratorFramework client,
+      CuratorEvent event) throws
+      Exception {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Delete event {}", event);
+    }
+    events.incrementAndGet();
+  }
+
+  /**
+   * Get the number of deletion events
+   * @return the count of events
+   */
+  public int getEventCount() {
+    return events.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java
new file mode 100644
index 0000000..3fa0c19
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java
@@ -0,0 +1,282 @@
+/*
+ * 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.hadoop.registry.server.services;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
+import org.apache.curator.ensemble.fixed.FixedEnsembleProvider;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.registry.client.impl.zk.BindingInformation;
+import org.apache.hadoop.registry.client.impl.zk.RegistryBindingSource;
+import org.apache.hadoop.registry.client.impl.zk.RegistryInternalConstants;
+import org.apache.hadoop.registry.client.impl.zk.RegistrySecurity;
+import org.apache.hadoop.registry.client.impl.zk.ZookeeperConfigOptions;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+
+/**
+ * This is a small, localhost Zookeeper service instance that is contained
+ * in a YARN service...it's been derived from Apache Twill.
+ *
+ * It implements {@link RegistryBindingSource} and provides binding information,
+ * <i>once started</i>. Until <code>start()</code> is called, the hostname &
+ * port may be undefined. Accordingly, the service raises an exception in this
+ * condition.
+ *
+ * If you wish to chain together a registry service with this one under
+ * the same <code>CompositeService</code>, this service must be added
+ * as a child first.
+ *
+ * It also sets the configuration parameter
+ * {@link RegistryConstants#KEY_REGISTRY_ZK_QUORUM}
+ * to its connection string. Any code with access to the service configuration
+ * can view it.
+ */
+@InterfaceStability.Evolving
+public class MicroZookeeperService
+    extends AbstractService
+    implements RegistryBindingSource, RegistryConstants,
+    ZookeeperConfigOptions,
+    MicroZookeeperServiceKeys{
+
+
+  private static final Logger
+      LOG = LoggerFactory.getLogger(MicroZookeeperService.class);
+
+  private File instanceDir;
+  private File dataDir;
+  private int tickTime;
+  private int port;
+  private String host;
+  private boolean secureServer;
+
+  private ServerCnxnFactory factory;
+  private BindingInformation binding;
+  private File confDir;
+  private StringBuilder diagnostics = new StringBuilder();
+
+  /**
+   * Create an instance
+   * @param name service name
+   */
+  public MicroZookeeperService(String name) {
+    super(name);
+  }
+
+  /**
+   * Get the connection string.
+   * @return the string
+   * @throws IllegalStateException if the connection is not yet valid
+   */
+  public String getConnectionString() {
+    Preconditions.checkState(factory != null, "service not started");
+    InetSocketAddress addr = factory.getLocalAddress();
+    return String.format("%s:%d", addr.getHostName(), addr.getPort());
+  }
+
+  /**
+   * Get the connection address
+   * @return the connection as an address
+   * @throws IllegalStateException if the connection is not yet valid
+   */
+  public InetSocketAddress getConnectionAddress() {
+    Preconditions.checkState(factory != null, "service not started");
+    return factory.getLocalAddress();
+  }
+
+  /**
+   * Create an inet socket addr from the local host + port number
+   * @param port port to use
+   * @return a (hostname, port) pair
+   * @throws UnknownHostException if the server cannot resolve the host
+   */
+  private InetSocketAddress getAddress(int port) throws UnknownHostException {
+    return new InetSocketAddress(host, port < 0 ? 0 : port);
+  }
+
+  /**
+   * Initialize the service, including choosing a path for the data
+   * @param conf configuration
+   * @throws Exception
+   */
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    port = conf.getInt(KEY_ZKSERVICE_PORT, 0);
+    tickTime = conf.getInt(KEY_ZKSERVICE_TICK_TIME,
+        ZooKeeperServer.DEFAULT_TICK_TIME);
+    String instancedirname = conf.getTrimmed(
+        KEY_ZKSERVICE_DIR, "");
+    host = conf.getTrimmed(KEY_ZKSERVICE_HOST, DEFAULT_ZKSERVICE_HOST);
+    if (instancedirname.isEmpty()) {
+      File testdir = new File(System.getProperty("test.dir", "target"));
+      instanceDir = new File(testdir, "zookeeper" + getName());
+    } else {
+      instanceDir = new File(instancedirname);
+      FileUtil.fullyDelete(instanceDir);
+    }
+    LOG.debug("Instance directory is {}", instanceDir);
+    mkdirStrict(instanceDir);
+    dataDir = new File(instanceDir, "data");
+    confDir = new File(instanceDir, "conf");
+    mkdirStrict(dataDir);
+    mkdirStrict(confDir);
+    super.serviceInit(conf);
+  }
+
+  /**
+   * Create a directory, ignoring if the dir is already there,
+   * and failing if a file or something else was at the end of that
+   * path
+   * @param dir dir to guarantee the existence of
+   * @throws IOException IO problems, or path exists but is not a dir
+   */
+  private void mkdirStrict(File dir) throws IOException {
+    if (!dir.mkdirs()) {
+      if (!dir.isDirectory()) {
+        throw new IOException("Failed to mkdir " + dir);
+      }
+    }
+  }
+
+  /**
+   * Append a formatted string to the diagnostics.
+   * <p>
+   * A newline is appended afterwards.
+   * @param text text including any format commands
+   * @param args arguments for the forma operation.
+   */
+  protected void addDiagnostics(String text, Object ... args) {
+    diagnostics.append(String.format(text, args)).append('\n');
+  }
+
+  /**
+   * Get the diagnostics info
+   * @return the diagnostics string built up
+   */
+  public String getDiagnostics() {
+    return diagnostics.toString();
+  }
+
+  /**
+   * set up security. this must be done prior to creating
+   * the ZK instance, as it sets up JAAS if that has not been done already.
+   *
+   * @return true if the cluster has security enabled.
+   */
+  public boolean setupSecurity() throws IOException {
+    Configuration conf = getConfig();
+    String jaasContext = conf.getTrimmed(KEY_REGISTRY_ZKSERVICE_JAAS_CONTEXT);
+    secureServer = StringUtils.isNotEmpty(jaasContext);
+    if (secureServer) {
+      RegistrySecurity.validateContext(jaasContext);
+      RegistrySecurity.bindZKToServerJAASContext(jaasContext);
+      // policy on failed auth
+      System.setProperty(PROP_ZK_ALLOW_FAILED_SASL_CLIENTS,
+        conf.get(KEY_ZKSERVICE_ALLOW_FAILED_SASL_CLIENTS,
+            "true"));
+
+      //needed so that you can use sasl: strings in the registry
+      System.setProperty(RegistryInternalConstants.ZOOKEEPER_AUTH_PROVIDER +".1",
+          RegistryInternalConstants.SASLAUTHENTICATION_PROVIDER);
+      String serverContext =
+          System.getProperty(PROP_ZK_SERVER_SASL_CONTEXT);
+      addDiagnostics("Server JAAS context s = %s", serverContext);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Startup: start ZK. It is only after this that
+   * the binding information is valid.
+   * @throws Exception
+   */
+  @Override
+  protected void serviceStart() throws Exception {
+
+    setupSecurity();
+
+    ZooKeeperServer zkServer = new ZooKeeperServer();
+    FileTxnSnapLog ftxn = new FileTxnSnapLog(dataDir, dataDir);
+    zkServer.setTxnLogFactory(ftxn);
+    zkServer.setTickTime(tickTime);
+
+    LOG.info("Starting Local Zookeeper service");
+    factory = ServerCnxnFactory.createFactory();
+    factory.configure(getAddress(port), -1);
+    factory.startup(zkServer);
+
+    String connectString = getConnectionString();
+    LOG.info("In memory ZK started at {}\n", connectString);
+
+    if (LOG.isDebugEnabled()) {
+      StringWriter sw = new StringWriter();
+      PrintWriter pw = new PrintWriter(sw);
+      zkServer.dumpConf(pw);
+      pw.flush();
+      LOG.debug(sw.toString());
+    }
+    binding = new BindingInformation();
+    binding.ensembleProvider = new FixedEnsembleProvider(connectString);
+    binding.description =
+        getName() + " reachable at \"" + connectString + "\"";
+
+    addDiagnostics(binding.description);
+    // finally: set the binding information in the config
+    getConfig().set(KEY_REGISTRY_ZK_QUORUM, connectString);
+  }
+
+  /**
+   * When the service is stopped, it deletes the data directory
+   * and its contents
+   * @throws Exception
+   */
+  @Override
+  protected void serviceStop() throws Exception {
+    if (factory != null) {
+      factory.shutdown();
+      factory = null;
+    }
+    if (dataDir != null) {
+      FileUtil.fullyDelete(dataDir);
+    }
+  }
+
+  @Override
+  public BindingInformation supplyBindingInformation() {
+    Preconditions.checkNotNull(binding,
+        "Service is not started: binding information undefined");
+    return binding;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperServiceKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperServiceKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperServiceKeys.java
new file mode 100644
index 0000000..f4f4976
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperServiceKeys.java
@@ -0,0 +1,69 @@
+/*
+ * 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.hadoop.registry.server.services;
+
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+
+/**
+ * Service keys for configuring the {@link MicroZookeeperService}.
+ * These are not used in registry clients or the RM-side service,
+ * so are kept separate.
+ */
+public interface MicroZookeeperServiceKeys {
+  public static final String ZKSERVICE_PREFIX =
+      RegistryConstants.REGISTRY_PREFIX + "zk.service.";
+  /**
+   * Key to define the JAAS context for the ZK service: {@value}.
+   */
+  public static final String KEY_REGISTRY_ZKSERVICE_JAAS_CONTEXT =
+      ZKSERVICE_PREFIX + "service.jaas.context";
+
+  /**
+   * ZK servertick time: {@value}
+   */
+  public static final String KEY_ZKSERVICE_TICK_TIME =
+      ZKSERVICE_PREFIX + "ticktime";
+
+  /**
+   * host to register on: {@value}.
+   */
+  public static final String KEY_ZKSERVICE_HOST = ZKSERVICE_PREFIX + "host";
+  /**
+   * Default host to serve on -this is <code>localhost</code> as it
+   * is the only one guaranteed to be available: {@value}.
+   */
+  public static final String DEFAULT_ZKSERVICE_HOST = "localhost";
+  /**
+   * port; 0 or below means "any": {@value}
+   */
+  public static final String KEY_ZKSERVICE_PORT = ZKSERVICE_PREFIX + "port";
+
+  /**
+   * Directory containing data: {@value}
+   */
+  public static final String KEY_ZKSERVICE_DIR = ZKSERVICE_PREFIX + "dir";
+
+  /**
+   * Should failed SASL clients be allowed: {@value}?
+   *
+   * Default is the ZK default: true
+   */
+  public static final String KEY_ZKSERVICE_ALLOW_FAILED_SASL_CLIENTS =
+      ZKSERVICE_PREFIX + "allow.failed.sasl.clients";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/RegistryAdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/RegistryAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/RegistryAdminService.java
new file mode 100644
index 0000000..693bb0b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/RegistryAdminService.java
@@ -0,0 +1,529 @@
+/*
+ * 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.hadoop.registry.server.services;
+
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.lang.StringUtils;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.service.ServiceStateException;
+import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.registry.client.exceptions.NoPathPermissionsException;
+import org.apache.hadoop.registry.client.exceptions.NoRecordException;
+import org.apache.hadoop.registry.client.impl.zk.RegistryBindingSource;
+import org.apache.hadoop.registry.client.impl.zk.RegistryOperationsService;
+import org.apache.hadoop.registry.client.impl.zk.RegistrySecurity;
+import org.apache.hadoop.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Administrator service for the registry. This is the one with
+ * permissions to create the base directories and those for users.
+ *
+ * It also includes support for asynchronous operations, so that
+ * zookeeper connectivity problems do not hold up the server code
+ * performing the actions.
+ *
+ * Any action queued via {@link #submit(Callable)} will be
+ * run asynchronously. The {@link #createDirAsync(String, List, boolean)}
+ * is an example of such an an action
+ *
+ * A key async action is the depth-first tree purge, which supports
+ * pluggable policies for deleting entries. The method
+ * {@link #purge(String, NodeSelector, PurgePolicy, BackgroundCallback)}
+ * implements the recursive purge operation —the class
+ * {{AsyncPurge}} provides the asynchronous scheduling of this.
+ */
+public class RegistryAdminService extends RegistryOperationsService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RegistryAdminService.class);
+  /**
+   * The ACL permissions for the user's homedir ACL.
+   */
+  public static final int USER_HOMEDIR_ACL_PERMISSIONS =
+        ZooDefs.Perms.READ | ZooDefs.Perms.WRITE
+      | ZooDefs.Perms.CREATE | ZooDefs.Perms.DELETE;
+
+  /**
+   * Executor for async operations
+   */
+  protected final ExecutorService executor;
+
+  /**
+   * Construct an instance of the service
+   * @param name service name
+   */
+  public RegistryAdminService(String name) {
+    this(name, null);
+  }
+
+  /**
+   * construct an instance of the service, using the
+   * specified binding source to bond to ZK
+   * @param name service name
+   * @param bindingSource provider of ZK binding information
+   */
+  public RegistryAdminService(String name,
+      RegistryBindingSource bindingSource) {
+    super(name, bindingSource);
+    executor = Executors.newCachedThreadPool(
+        new ThreadFactory() {
+          private AtomicInteger counter = new AtomicInteger(1);
+
+          @Override
+          public Thread newThread(Runnable r) {
+            return new Thread(r,
+                "RegistryAdminService " + counter.getAndIncrement());
+          }
+        });
+  }
+
+  /**
+   * Stop the service: halt the executor.
+   * @throws Exception exception.
+   */
+  @Override
+  protected void serviceStop() throws Exception {
+    stopExecutor();
+    super.serviceStop();
+  }
+
+  /**
+   * Stop the executor if it is not null.
+   * This uses {@link ExecutorService#shutdownNow()}
+   * and so does not block until they have completed.
+   */
+  protected synchronized void stopExecutor() {
+    if (executor != null) {
+      executor.shutdownNow();
+    }
+  }
+
+  /**
+   * Get the executor
+   * @return the executor
+   */
+  protected ExecutorService getExecutor() {
+    return executor;
+  }
+
+  /**
+   * Submit a callable
+   * @param callable callable
+   * @param <V> type of the final get
+   * @return a future to wait on
+   */
+  public <V> Future<V> submit(Callable<V> callable) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Submitting {}", callable);
+    }
+    return getExecutor().submit(callable);
+  }
+
+  /**
+   * Asynchronous operation to create a directory
+   * @param path path
+   * @param acls ACL list
+   * @param createParents flag to indicate parent dirs should be created
+   * as needed
+   * @return the future which will indicate whether or not the operation
+   * succeeded —and propagate any exceptions
+   * @throws IOException
+   */
+  public Future<Boolean> createDirAsync(final String path,
+      final List<ACL> acls,
+      final boolean createParents) throws IOException {
+    return submit(new Callable<Boolean>() {
+      @Override
+      public Boolean call() throws Exception {
+        return maybeCreate(path, CreateMode.PERSISTENT,
+            acls, createParents);
+      }
+    });
+  }
+
+  /**
+   * Init operation sets up the system ACLs.
+   * @param conf configuration of the service
+   * @throws Exception
+   */
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    RegistrySecurity registrySecurity = getRegistrySecurity();
+    if (registrySecurity.isSecureRegistry()) {
+      ACL sasl = registrySecurity.createSaslACLFromCurrentUser(ZooDefs.Perms.ALL);
+      registrySecurity.addSystemACL(sasl);
+      LOG.info("Registry System ACLs:",
+          RegistrySecurity.aclsToString(
+          registrySecurity.getSystemACLs()));
+    }
+  }
+
+  /**
+   * Start the service, including creating base directories with permissions
+   * @throws Exception
+   */
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    // create the root directories
+    try {
+      createRootRegistryPaths();
+    } catch (NoPathPermissionsException e) {
+
+      String message = String.format(Locale.ENGLISH,
+          "Failed to create root paths {%s};" +
+          "\ndiagnostics={%s}" +
+          "\ncurrent registry is:" +
+          "\n{%s}",
+          e,
+          bindingDiagnosticDetails(),
+          dumpRegistryRobustly(true));
+
+      LOG.error(" Failure {}", e, e);
+      LOG.error(message);
+
+      // TODO: this is something temporary to deal with the problem
+      // that jenkins is failing this test
+      throw new NoPathPermissionsException(e.getPath().toString(), message, e);
+    }
+  }
+
+  /**
+   * Create the initial registry paths
+   * @throws IOException any failure
+   */
+  @VisibleForTesting
+  public void createRootRegistryPaths() throws IOException {
+
+    List<ACL> systemACLs = getRegistrySecurity().getSystemACLs();
+    LOG.info("System ACLs {}",
+        RegistrySecurity.aclsToString(systemACLs));
+    maybeCreate("", CreateMode.PERSISTENT, systemACLs, false);
+    maybeCreate(PATH_USERS, CreateMode.PERSISTENT,
+        systemACLs, false);
+    maybeCreate(PATH_SYSTEM_SERVICES,
+        CreateMode.PERSISTENT,
+        systemACLs, false);
+  }
+
+  /**
+   * Get the path to a user's home dir
+   * @param username username
+   * @return a path for services underneath
+   */
+  protected String homeDir(String username) {
+    return RegistryUtils.homePathForUser(username);
+  }
+
+  /**
+   * Set up the ACL for the user.
+   * <b>Important: this must run client-side as it needs
+   * to know the id:pass tuple for a user</b>
+   * @param username user name
+   * @param perms permissions
+   * @return an ACL list
+   * @throws IOException ACL creation/parsing problems
+   */
+  public List<ACL> aclsForUser(String username, int perms) throws IOException {
+    List<ACL> clientACLs = getClientAcls();
+    RegistrySecurity security = getRegistrySecurity();
+    if (security.isSecureRegistry()) {
+      clientACLs.add(security.createACLfromUsername(username, perms));
+    }
+    return clientACLs;
+  }
+
+  /**
+   * Start an async operation to create the home path for a user
+   * if it does not exist
+   * @param shortname username, without any @REALM in kerberos
+   * @return the path created
+   * @throws IOException any failure while setting up the operation
+   *
+   */
+  public Future<Boolean> initUserRegistryAsync(final String shortname)
+      throws IOException {
+
+    String homeDir = homeDir(shortname);
+    if (!exists(homeDir)) {
+      // create the directory. The user does not
+      return createDirAsync(homeDir,
+          aclsForUser(shortname,
+              USER_HOMEDIR_ACL_PERMISSIONS),
+          false);
+    }
+    return null;
+  }
+
+  /**
+   * Create the home path for a user if it does not exist.
+   *
+   * This uses {@link #initUserRegistryAsync(String)} and then waits for the
+   * result ... the code path is the same as the async operation; this just
+   * picks up and relays/converts exceptions
+   * @param username username
+   * @return the path created
+   * @throws IOException any failure
+   *
+   */
+  public String initUserRegistry(final String username)
+      throws IOException {
+
+    try {
+      Future<Boolean> future = initUserRegistryAsync(username);
+      future.get();
+    } catch (InterruptedException e) {
+      throw (InterruptedIOException)
+          (new InterruptedIOException(e.toString()).initCause(e));
+    } catch (ExecutionException e) {
+      Throwable cause = e.getCause();
+      if (cause instanceof IOException) {
+        throw (IOException) (cause);
+      } else {
+        throw new IOException(cause.toString(), cause);
+      }
+    }
+
+    return homeDir(username);
+  }
+
+  /**
+   * Method to validate the validity of the kerberos realm.
+   * <ul>
+   *   <li>Insecure: not needed.</li>
+   *   <li>Secure: must have been determined.</li>
+   * </ul>
+   */
+  protected void verifyRealmValidity() throws ServiceStateException {
+    if (isSecure()) {
+      String realm = getRegistrySecurity().getKerberosRealm();
+      if (StringUtils.isEmpty(realm)) {
+        throw new ServiceStateException("Cannot determine service realm");
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Started Registry operations in realm {}", realm);
+      }
+    }
+  }
+
+  /**
+   * Policy to purge entries
+   */
+  public enum PurgePolicy {
+    PurgeAll,
+    FailOnChildren,
+    SkipOnChildren
+  }
+
+  /**
+   * Recursive operation to purge all matching records under a base path.
+   * <ol>
+   *   <li>Uses a depth first search</li>
+   *   <li>A match is on ID and persistence policy, or, if policy==-1, any match</li>
+   *   <li>If a record matches then it is deleted without any child searches</li>
+   *   <li>Deletions will be asynchronous if a callback is provided</li>
+   * </ol>
+   *
+   * The code is designed to be robust against parallel deletions taking place;
+   * in such a case it will stop attempting that part of the tree. This
+   * avoid the situation of more than 1 purge happening in parallel and
+   * one of the purge operations deleteing the node tree above the other.
+   * @param path base path
+   * @param selector selector for the purge policy
+   * @param purgePolicy what to do if there is a matching record with children
+   * @param callback optional curator callback
+   * @return the number of delete operations perfomed. As deletes may be for
+   * everything under a path, this may be less than the number of records
+   * actually deleted
+   * @throws IOException problems
+   * @throws PathIsNotEmptyDirectoryException if an entry cannot be deleted
+   * as it has children and the purge policy is FailOnChildren
+   */
+  @VisibleForTesting
+  public int purge(String path,
+      NodeSelector selector,
+      PurgePolicy purgePolicy,
+      BackgroundCallback callback) throws IOException {
+
+
+    boolean toDelete = false;
+    // look at self to see if it has a service record
+    Map<String, RegistryPathStatus> childEntries;
+    Collection<RegistryPathStatus> entries;
+    try {
+      // list this path's children
+      childEntries = RegistryUtils.statChildren(this, path);
+      entries = childEntries.values();
+    } catch (PathNotFoundException e) {
+      // there's no record here, it may have been deleted already.
+      // exit
+      return 0;
+    }
+
+    try {
+      RegistryPathStatus registryPathStatus = stat(path);
+      ServiceRecord serviceRecord = resolve(path);
+      // there is now an entry here.
+      toDelete = selector.shouldSelect(path, registryPathStatus, serviceRecord);
+    } catch (EOFException ignored) {
+      // ignore
+    } catch (InvalidRecordException ignored) {
+      // ignore
+    } catch (NoRecordException ignored) {
+      // ignore
+    } catch (PathNotFoundException e) {
+      // there's no record here, it may have been deleted already.
+      // exit
+      return 0;
+    }
+
+    if (toDelete && !entries.isEmpty()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Match on record @ {} with children ", path);
+      }
+      // there's children
+      switch (purgePolicy) {
+        case SkipOnChildren:
+          // don't do the deletion... continue to next record
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Skipping deletion");
+          }
+          toDelete = false;
+          break;
+        case PurgeAll:
+          // mark for deletion
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Scheduling for deletion with children");
+          }
+          toDelete = true;
+          entries = new ArrayList<RegistryPathStatus>(0);
+          break;
+        case FailOnChildren:
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Failing deletion operation");
+          }
+          throw new PathIsNotEmptyDirectoryException(path);
+      }
+    }
+
+    int deleteOps = 0;
+    if (toDelete) {
+      try {
+        zkDelete(path, true, callback);
+      } catch (PathNotFoundException e) {
+        // sign that the path was deleted during the operation.
+        // this is a no-op, and all children can be skipped
+        return deleteOps;
+      }
+      deleteOps++;
+    }
+
+    // now go through the children
+    for (RegistryPathStatus status : entries) {
+      String childname = status.path;
+      String childpath = RegistryPathUtils.join(path, childname);
+      deleteOps += purge(childpath,
+          selector,
+          purgePolicy,
+          callback);
+    }
+
+    return deleteOps;
+  }
+
+  /**
+   * Comparator used for purge logic
+   */
+  public interface NodeSelector {
+
+    boolean shouldSelect(String path,
+        RegistryPathStatus registryPathStatus,
+        ServiceRecord serviceRecord);
+  }
+
+  /**
+   * An async registry purge action taking
+   * a selector which decides what to delete
+   */
+  public class AsyncPurge implements Callable<Integer> {
+
+    private final BackgroundCallback callback;
+    private final NodeSelector selector;
+    private final String path;
+    private final PurgePolicy purgePolicy;
+
+    public AsyncPurge(String path,
+        NodeSelector selector,
+        PurgePolicy purgePolicy,
+        BackgroundCallback callback) {
+      this.callback = callback;
+      this.selector = selector;
+      this.path = path;
+      this.purgePolicy = purgePolicy;
+    }
+
+    @Override
+    public Integer call() throws Exception {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Executing {}", this);
+      }
+      return purge(path,
+          selector,
+          purgePolicy,
+          callback);
+    }
+
+    @Override
+    public String toString() {
+      return String.format(
+          "Record purge under %s with selector %s",
+          path, selector);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java
new file mode 100644
index 0000000..85d24b3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+/**
+ * Basic services for the YARN registry
+ * <ul>
+ *   <li>The {@link org.apache.hadoop.registry.server.services.RegistryAdminService}</ol>
+ *   extends the shared Yarn Registry client with registry setup and
+ *   (potentially asynchronous) administrative actions.
+ *   </li>
+ *   <li>
+ *     The {@link org.apache.hadoop.registry.server.services.MicroZookeeperService}
+ *     is a transient Zookeeper instance bound to the YARN service lifecycle.
+ *     It is suitable for testing.
+ *   </li>
+ *   <li>
+ *     The {@link org.apache.hadoop.registry.server.services.AddingCompositeService}
+ *     extends the standard YARN composite service by making its add and remove
+ *     methods public. It is a utility service used in parts of the codebase
+ *   </li>
+ *
+ * </ul>
+ *
+ */
+package org.apache.hadoop.registry.server.services;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/resources/.keep
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/resources/.keep b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/resources/.keep
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/tla/yarnregistry.tla
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/tla/yarnregistry.tla b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/tla/yarnregistry.tla
new file mode 100644
index 0000000..1c19ade
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/tla/yarnregistry.tla
@@ -0,0 +1,538 @@
+---------------------------- MODULE yarnregistry ----------------------------
+
+EXTENDS FiniteSets, Sequences, Naturals, TLC
+
+
+(*
+ * 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.
+ *)
+
+(*
+
+============================================================================
+
+This defines the YARN registry in terms of operations on sets of records.
+
+Every registry entry is represented as a record containing both the path and the data.
+
+It assumes that
+
+1. operations on this set are immediate.
+2. selection operations (such as \A and \E are atomic)
+3. changes are immediately visible to all other users of the registry.
+4. This clearly implies that changes are visible in the sequence in which they happen.
+
+A multi-server Zookeeper-based registry may not meet all those assumptions
+
+1. changes may take time to propagate across the ZK quorum, hence changes cannot
+be considered immediate from the perspective of other registry clients.
+(assumptions (1) and (3)).
+
+2. Selection operations may not be atomic. (assumption (2)).
+
+Operations will still happen in the order received by the elected ZK master
+
+A stricter definition would try to state that all operations are eventually
+true excluding other changes happening during a sequence of action.
+This is left as an excercise for the reader.
+
+The specification also omits all coverage of the permissions policy.
+*)
+
+
+
+CONSTANTS
+    PathChars,     \* the set of valid characters in a path
+    Paths,         \* the set of all possible valid paths
+    Data,          \* the set of all possible sequences of bytes
+    Address,       \* the set of all possible address n-tuples
+    Addresses,     \* the set of all possible address instances
+    Endpoints ,    \* the set of all possible endpoints
+    PersistPolicies,\* the set of persistence policies
+    ServiceRecords, \* all service records
+    Registries,     \* the set of all possile registries
+    BindActions,     \* all possible put actions
+    DeleteActions,  \* all possible delete actions
+    PurgeActions,   \* all possible purge actions
+    MknodeActions    \* all possible mkdir actions
+
+
+
+(* the registry*)
+VARIABLE registry
+
+(* Sequence of actions to apply to the registry *)
+VARIABLE actions
+
+----------------------------------------------------------------------------------------
+(* Tuple of all variables.  *)
+
+
+vars == << registry, actions >>
+
+
+----------------------------------------------------------------------------------------
+
+
+
+
+(* Persistence policy *)
+PersistPolicySet == {
+    "",                      \* Undefined; field not present. PERMANENT is implied.
+    "permanent",            \* persists until explicitly removed
+    "application",          \* persists until the application finishes
+    "application-attempt",  \* persists until the application attempt finishes
+    "container"             \* persists until the container finishes
+  }
+
+(* Type invariants. *)
+TypeInvariant ==
+    /\ \A p \in PersistPolicies: p \in PersistPolicySet
+
+
+
+----------------------------------------------------------------------------------------
+
+
+
+(*
+
+An Entry is defined as a path, and the actual
+data which it contains.
+
+By including the path in an entry, we avoid having to define some
+function mapping Path -> entry.  Instead a registry can be defined as a
+set of RegistryEntries matching the validity critera.
+
+*)
+
+RegistryEntry == [
+    \* The path to the entry
+    path: Paths,
+
+    \* the data in the entry
+    data: Data
+    ]
+
+
+(*
+    An endpoint in a service record
+*)
+Endpoint == [
+    \* API of the endpoint: some identifier
+    api: STRING,
+
+    \* A list of address n-tuples
+    addresses: Addresses
+]
+
+(* Attributes are the set of all string to string mappings *)
+
+Attributes == [
+STRING |-> STRING
+]
+
+(*
+    A service record
+*)
+ServiceRecord == [
+    \* ID -used when applying the persistence policy
+    yarn_id: STRING,
+
+    \* the persistence policy
+    yarn_persistence: PersistPolicySet,
+
+    \*A description
+    description: STRING,
+
+    \* A set of endpoints
+    external: Endpoints,
+
+    \* Endpoints intended for use internally
+    internal: Endpoints,
+
+    \* Attributes are a function
+    attributes: Attributes
+]
+
+
+----------------------------------------------------------------------------------------
+
+(* Action Records *)
+
+putAction == [
+    type: "put",
+    record: ServiceRecord
+]
+
+deleteAction == [
+    type: "delete",
+    path: STRING,
+    recursive: BOOLEAN
+]
+
+purgeAction == [
+    type: "purge",
+    path: STRING,
+    persistence: PersistPolicySet
+]
+
+mkNodeAction == [
+    type: "mknode",
+    path: STRING,
+    parents: BOOLEAN
+]
+
+
+----------------------------------------------------------------------------------------
+
+(*
+
+ Path operations
+
+*)
+
+(*
+Parent is defined for non empty sequences
+ *)
+
+parent(path) == SubSeq(path, 1, Len(path)-1)
+
+isParent(path, c) == path = parent(c)
+
+----------------------------------------------------------------------------------------
+(*
+Registry Access Operations
+*)
+
+(*
+Lookup all entries in a registry with a matching path
+*)
+
+resolve(Registry, path) == \A entry \in Registry: entry.path = path
+
+(*
+A path exists in the registry iff there is an entry with that path
+*)
+
+exists(Registry, path) == resolve(Registry, path) /= {}
+
+(*
+A parent entry, or an empty set if there is none
+*)
+parentEntry(Registry, path) == resolve(Registry, parent(path))
+
+(*
+A root path is the empty sequence
+*)
+isRootPath(path) == path = <<>>
+
+(*
+The root entry is the entry whose path is the root path
+*)
+isRootEntry(entry) == entry.path = <<>>
+
+
+(*
+A path p is an ancestor of another path d if they are different, and the path d
+starts with path p
+*)
+
+isAncestorOf(path, d) ==
+    /\ path /= d
+    /\ \E k : SubSeq(d, 0, k) = path
+
+
+ancestorPathOf(path) ==
+    \A a \in Paths: isAncestorOf(a, path)
+
+(*
+The set of all children of a path in the registry
+*)
+
+children(R, path) == \A c \in R: isParent(path, c.path)
+
+(*
+A path has children if the children() function does not return the empty set
+*)
+hasChildren(R, path) == children(R, path) /= {}
+
+(*
+Descendant: a child of a path or a descendant of a child of a path
+*)
+
+descendants(R, path) == \A e \in R: isAncestorOf(path, e.path)
+
+(*
+Ancestors: all entries in the registry whose path is an entry of the path argument
+*)
+ancestors(R, path) == \A e \in R: isAncestorOf(e.path, path)
+
+(*
+The set of entries that are a path and its descendants
+*)
+pathAndDescendants(R, path) ==
+    \/ \A e \in R: isAncestorOf(path, e.path)
+    \/ resolve(R, path)
+
+
+(*
+For validity, all entries must match the following criteria
+ *)
+
+validRegistry(R) ==
+        \* there can be at most one entry for a path.
+        /\ \A e \in R: Cardinality(resolve(R, e.path)) = 1
+
+        \* There's at least one root entry
+        /\ \E e \in R: isRootEntry(e)
+
+        \* an entry must be the root entry or have a parent entry
+        /\ \A e \in R: isRootEntry(e) \/ exists(R, parent(e.path))
+
+        \* If the entry has data, it must be a service record
+        /\ \A e \in R: (e.data = << >> \/ e.data \in ServiceRecords)
+
+
+----------------------------------------------------------------------------------------
+(*
+Registry Manipulation
+*)
+
+(*
+An entry can be put into the registry iff
+its parent is present or it is the root entry
+*)
+canBind(R, e) ==
+    isRootEntry(e) \/ exists(R, parent(e.path))
+
+(*
+'bind() adds/replaces an entry if permitted
+*)
+
+bind(R, e) ==
+    /\ canBind(R, e)
+    /\ R' = (R \ resolve(R, e.path)) \union {e}
+
+
+(*
+mknode() adds a new empty entry where there was none before, iff
+-the parent exists
+-it meets the requirement for being "bindable"
+*)
+
+mknodeSimple(R, path) ==
+    LET record == [ path |-> path, data |-> <<>>  ]
+    IN  \/ exists(R, path)
+        \/ (exists(R, parent(path))  /\ canBind(R, record) /\ (R' = R \union {record} ))
+
+
+(*
+For all parents, the mknodeSimpl() criteria must apply.
+This could be defined recursively, though as TLA+ does not support recursion,
+an alternative is required
+
+
+Because this specification is declaring the final state of a operation, not
+the implemental, all that is needed is to describe those parents.
+
+It declares that the mkdirSimple state applies to the path and all its parents in the set R'
+
+*)
+mknodeWithParents(R, path) ==
+    /\ \A p2 \in ancestors(R, path) : mknodeSimple(R, p2)
+    /\ mknodeSimple(R, path)
+
+
+mknode(R, path, recursive) ==
+   IF recursive THEN mknodeWithParents(R, path) ELSE mknodeSimple(R, path)
+
+(*
+Deletion is set difference on any existing entries
+*)
+
+simpleDelete(R, path) ==
+    /\ ~isRootPath(path)
+    /\ children(R, path) = {}
+    /\ R' = R \ resolve(R, path)
+
+(*
+Recursive delete: neither the path or its descendants exists in the new registry
+*)
+
+recursiveDelete(R, path) ==
+       \* Root path: the new registry is the initial registry again
+    /\ isRootPath(path) => R' = { [ path |-> <<>>, data |-> <<>> ] }
+       \*  Any other entry: the new registry is a set with any existing
+       \* entry for that path is removed, and the new entry added
+    /\ ~isRootPath(path) => R' = R \ ( resolve(R, path) \union descendants(R, path))
+
+
+(*
+Delete operation which chooses the recursiveness policy based on an argument
+*)
+
+delete(R, path, recursive) ==
+    IF recursive THEN recursiveDelete(R, path) ELSE simpleDelete(R, path)
+
+
+(*
+Purge ensures that all entries under a path with the matching ID and policy are not there
+afterwards
+*)
+
+purge(R, path, id, persistence) ==
+    /\ (persistence \in PersistPolicySet)
+    /\ \A p2 \in pathAndDescendants(R, path) :
+         (p2.attributes["yarn:id"] = id /\ p2.attributes["yarn:persistence"] = persistence)
+         => recursiveDelete(R, p2.path)
+
+(*
+resolveRecord() resolves the record at a path or fails.
+
+It relies on the fact that if the cardinality of a set is 1, then the CHOOSE operator
+is guaranteed to return the single entry of that set, iff the choice predicate holds.
+
+Using a predicate of TRUE, it always succeeds, so this function selects
+the sole entry of the resolve operation.
+*)
+
+resolveRecord(R, path) ==
+    LET l == resolve(R, path) IN
+        /\ Cardinality(l) = 1
+        /\ CHOOSE e \in l : TRUE
+
+(*
+The specific action of putting an entry into a record includes validating the record
+*)
+
+validRecordToBind(path, record) ==
+      \* The root entry must have permanent persistence
+     isRootPath(path) => (record.attributes["yarn:persistence"] = "permanent"
+     \/ record.attributes["yarn:persistence"] = "")
+
+
+(*
+Binding a service record involves validating it then putting it in the registry
+marshalled as the data in the entry
+ *)
+bindRecord(R, path, record) ==
+    /\ validRecordToBind(path, record)
+    /\ bind(R, [path |-> path, data |-> record])
+
+
+----------------------------------------------------------------------------------------
+
+
+
+(*
+The action queue can only contain one of the sets of action types, and
+by giving each a unique name, those sets are guaranteed to be disjoint
+*)
+ QueueInvariant ==
+    /\ \A a \in actions:
+        \/ (a \in BindActions /\ a.type="bind")
+        \/ (a \in DeleteActions /\ a.type="delete")
+        \/ (a \in PurgeActions /\ a.type="purge")
+        \/ (a \in MknodeActions /\ a.type="mknode")
+
+
+(*
+Applying queued actions
+*)
+
+applyAction(R, a) ==
+    \/ (a \in BindActions /\ bindRecord(R, a.path, a.record) )
+    \/ (a \in MknodeActions /\ mknode(R, a.path, a.recursive) )
+    \/ (a \in DeleteActions /\ delete(R, a.path, a.recursive) )
+    \/ (a \in PurgeActions /\ purge(R, a.path, a.id, a.persistence))
+
+
+(*
+Apply the first action in a list and then update the actions
+*)
+applyFirstAction(R, a) ==
+    /\ actions /= <<>>
+    /\ applyAction(R, Head(a))
+    /\ actions' = Tail(a)
+
+
+Next == applyFirstAction(registry, actions)
+
+(*
+All submitted actions must eventually be applied.
+*)
+
+
+Liveness == <>( actions = <<>> )
+
+
+(*
+The initial state of a registry has the root entry.
+*)
+
+InitialRegistry == registry = {
+  [ path |-> <<>>, data |-> <<>> ]
+}
+
+
+(*
+The valid state of the "registry" variable is defined as
+Via the validRegistry predicate
+*)
+
+ValidRegistryState == validRegistry(registry)
+
+
+
+(*
+The initial state of the system
+*)
+InitialState ==
+    /\ InitialRegistry
+    /\ ValidRegistryState
+    /\ actions = <<>>
+
+
+(*
+The registry has an initial state, the series of state changes driven by the actions,
+and the requirement that it does act on those actions.
+*)
+RegistrySpec ==
+    /\ InitialState
+    /\ [][Next]_vars
+    /\ Liveness
+
+
+----------------------------------------------------------------------------------------
+
+(*
+Theorem: For all operations from that initial state, the registry state is still valid
+*)
+THEOREM InitialState => [] ValidRegistryState
+
+(*
+Theorem: for all operations from that initial state, the type invariants hold
+*)
+THEOREM InitialState => [] TypeInvariant
+
+(*
+Theorem: the queue invariants hold
+*)
+THEOREM InitialState => [] QueueInvariant
+
+=============================================================================

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/AbstractRegistryTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/AbstractRegistryTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/AbstractRegistryTest.java
new file mode 100644
index 0000000..5b34f60
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/AbstractRegistryTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.hadoop.registry;
+
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.server.integration.RMRegistryOperationsService;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+/**
+ * Abstract registry tests .. inits the field {@link #registry}
+ * before the test with an instance of {@link RMRegistryOperationsService};
+ * and {@link #operations} with the same instance cast purely
+ * to the type {@link RegistryOperations}.
+ *
+ */
+public class AbstractRegistryTest extends AbstractZKRegistryTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractRegistryTest.class);
+  protected RMRegistryOperationsService registry;
+  protected RegistryOperations operations;
+
+  @Before
+  public void setupRegistry() throws IOException {
+    registry = new RMRegistryOperationsService("yarnRegistry");
+    operations = registry;
+    registry.init(createRegistryConfiguration());
+    registry.start();
+    operations.delete("/", true);
+    registry.createRootRegistryPaths();
+    addToTeardown(registry);
+  }
+
+  /**
+   * Create a service entry with the sample endpoints, and put it
+   * at the destination
+   * @param path path
+   * @param createFlags flags
+   * @return the record
+   * @throws IOException on a failure
+   */
+  protected ServiceRecord putExampleServiceEntry(String path, int createFlags) throws
+      IOException,
+      URISyntaxException {
+    return putExampleServiceEntry(path, createFlags, PersistencePolicies.PERMANENT);
+  }
+
+  /**
+   * Create a service entry with the sample endpoints, and put it
+   * at the destination
+   * @param path path
+   * @param createFlags flags
+   * @return the record
+   * @throws IOException on a failure
+   */
+  protected ServiceRecord putExampleServiceEntry(String path,
+      int createFlags,
+      String persistence)
+      throws IOException, URISyntaxException {
+    ServiceRecord record = buildExampleServiceEntry(persistence);
+
+    registry.mknode(RegistryPathUtils.parentOf(path), true);
+    operations.bind(path, record, createFlags);
+    return record;
+  }
+
+  /**
+   * Assert a path exists
+   * @param path path in the registry
+   * @throws IOException
+   */
+  public void assertPathExists(String path) throws IOException {
+    operations.stat(path);
+  }
+
+  /**
+   * assert that a path does not exist
+   * @param path path in the registry
+   * @throws IOException
+   */
+  public void assertPathNotFound(String path) throws IOException {
+    try {
+      operations.stat(path);
+      fail("Path unexpectedly found: " + path);
+    } catch (PathNotFoundException e) {
+
+    }
+  }
+
+  /**
+   * Assert that a path resolves to a service record
+   * @param path path in the registry
+   * @throws IOException
+   */
+  public void assertResolves(String path) throws IOException {
+    operations.resolve(path);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/AbstractZKRegistryTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/AbstractZKRegistryTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/AbstractZKRegistryTest.java
new file mode 100644
index 0000000..bcff622
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/AbstractZKRegistryTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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.hadoop.registry;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.registry.server.services.AddingCompositeService;
+import org.apache.hadoop.registry.server.services.MicroZookeeperService;
+import org.apache.hadoop.registry.server.services.MicroZookeeperServiceKeys;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+
+public class AbstractZKRegistryTest extends RegistryTestHelper {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractZKRegistryTest.class);
+
+  private static final AddingCompositeService servicesToTeardown =
+      new AddingCompositeService("teardown");
+  // static initializer guarantees it is always started
+  // ahead of any @BeforeClass methods
+  static {
+    servicesToTeardown.init(new Configuration());
+    servicesToTeardown.start();
+  }
+
+  @Rule
+  public final Timeout testTimeout = new Timeout(10000);
+
+  @Rule
+  public TestName methodName = new TestName();
+
+  protected static void addToTeardown(Service svc) {
+    servicesToTeardown.addService(svc);
+  }
+
+  @AfterClass
+  public static void teardownServices() throws IOException {
+    describe(LOG, "teardown of static services");
+    servicesToTeardown.close();
+  }
+
+  protected static MicroZookeeperService zookeeper;
+
+
+  @BeforeClass
+  public static void createZKServer() throws Exception {
+    File zkDir = new File("target/zookeeper");
+    FileUtils.deleteDirectory(zkDir);
+    assertTrue(zkDir.mkdirs());
+    zookeeper = new MicroZookeeperService("InMemoryZKService");
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(MicroZookeeperServiceKeys.KEY_ZKSERVICE_DIR, zkDir.getAbsolutePath());
+    zookeeper.init(conf);
+    zookeeper.start();
+    addToTeardown(zookeeper);
+  }
+
+  /**
+   * give our thread a name
+   */
+  @Before
+  public void nameThread() {
+    Thread.currentThread().setName("JUnit");
+  }
+
+  /**
+   * Returns the connection string to use
+   *
+   * @return connection string
+   */
+  public String getConnectString() {
+    return zookeeper.getConnectionString();
+  }
+
+  public YarnConfiguration createRegistryConfiguration() {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setInt(RegistryConstants.KEY_REGISTRY_ZK_CONNECTION_TIMEOUT, 1000);
+    conf.setInt(RegistryConstants.KEY_REGISTRY_ZK_RETRY_INTERVAL, 500);
+    conf.setInt(RegistryConstants.KEY_REGISTRY_ZK_RETRY_TIMES, 10);
+    conf.setInt(RegistryConstants.KEY_REGISTRY_ZK_RETRY_CEILING, 10);
+    conf.set(RegistryConstants.KEY_REGISTRY_ZK_QUORUM,
+        zookeeper.getConnectionString());
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/088ae9c5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/RegistryTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/RegistryTestHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/RegistryTestHelper.java
new file mode 100644
index 0000000..38cc2cb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/RegistryTestHelper.java
@@ -0,0 +1,401 @@
+/*
+ * 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.hadoop.registry;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
+import org.apache.hadoop.registry.client.types.AddressTypes;
+import org.apache.hadoop.registry.client.types.Endpoint;
+import org.apache.hadoop.registry.client.types.ProtocolTypes;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
+import org.apache.hadoop.registry.secure.AbstractSecureRegistryTest;
+import org.apache.zookeeper.common.PathUtils;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.Subject;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.inetAddrEndpoint;
+import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.ipcEndpoint;
+import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.restEndpoint;
+import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.tuple;
+import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.webEndpoint;
+
+/**
+ * This is a set of static methods to aid testing the registry operations.
+ * The methods can be imported statically —or the class used as a base
+ * class for tests.
+ */
+public class RegistryTestHelper extends Assert {
+  public static final String SC_HADOOP = "org-apache-hadoop";
+  public static final String USER = "devteam/";
+  public static final String NAME = "hdfs";
+  public static final String API_WEBHDFS = "org_apache_hadoop_namenode_webhdfs";
+  public static final String API_HDFS = "org_apache_hadoop_namenode_dfs";
+  public static final String USERPATH = RegistryConstants.PATH_USERS + USER;
+  public static final String PARENT_PATH = USERPATH + SC_HADOOP + "/";
+  public static final String ENTRY_PATH = PARENT_PATH + NAME;
+  public static final String NNIPC = "nnipc";
+  public static final String IPC2 = "IPC2";
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RegistryTestHelper.class);
+  public static final String KTUTIL = "ktutil";
+  private static final RegistryUtils.ServiceRecordMarshal recordMarshal =
+      new RegistryUtils.ServiceRecordMarshal();
+
+  /**
+   * Assert the path is valid by ZK rules
+   * @param path path to check
+   */
+  public static void assertValidZKPath(String path) {
+    try {
+      PathUtils.validatePath(path);
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Invalid Path " + path + ": " + e, e);
+    }
+  }
+
+  /**
+   * Assert that a string is not empty (null or "")
+   * @param message message to raise if the string is empty
+   * @param check string to check
+   */
+  public static void assertNotEmpty(String message, String check) {
+    if (StringUtils.isEmpty(check)) {
+      fail(message);
+    }
+  }
+
+  /**
+   * Assert that a string is empty (null or "")
+   * @param check string to check
+   */
+  public static void assertNotEmpty(String check) {
+    if (StringUtils.isEmpty(check)) {
+      fail("Empty string");
+    }
+  }
+
+  /**
+   * Log the details of a login context
+   * @param name name to assert that the user is logged in as
+   * @param loginContext the login context
+   */
+  public static void logLoginDetails(String name,
+      LoginContext loginContext) {
+    assertNotNull("Null login context", loginContext);
+    Subject subject = loginContext.getSubject();
+    LOG.info("Logged in as {}:\n {}", name, subject);
+  }
+
+  /**
+   * Set the JVM property to enable Kerberos debugging
+   */
+  public static void enableKerberosDebugging() {
+    System.setProperty(AbstractSecureRegistryTest.SUN_SECURITY_KRB5_DEBUG,
+        "true");
+  }
+  /**
+   * Set the JVM property to enable Kerberos debugging
+   */
+  public static void disableKerberosDebugging() {
+    System.setProperty(AbstractSecureRegistryTest.SUN_SECURITY_KRB5_DEBUG,
+        "false");
+  }
+
+  /**
+   * General code to validate bits of a component/service entry built iwth
+   * {@link #addSampleEndpoints(ServiceRecord, String)}
+   * @param record instance to check
+   */
+  public static void validateEntry(ServiceRecord record) {
+    assertNotNull("null service record", record);
+    List<Endpoint> endpoints = record.external;
+    assertEquals(2, endpoints.size());
+
+    Endpoint webhdfs = findEndpoint(record, API_WEBHDFS, true, 1, 1);
+    assertEquals(API_WEBHDFS, webhdfs.api);
+    assertEquals(AddressTypes.ADDRESS_URI, webhdfs.addressType);
+    assertEquals(ProtocolTypes.PROTOCOL_REST, webhdfs.protocolType);
+    List<List<String>> addressList = webhdfs.addresses;
+    List<String> url = addressList.get(0);
+    String addr = url.get(0);
+    assertTrue(addr.contains("http"));
+    assertTrue(addr.contains(":8020"));
+
+    Endpoint nnipc = findEndpoint(record, NNIPC, false, 1,2);
+    assertEquals("wrong protocol in " + nnipc, ProtocolTypes.PROTOCOL_THRIFT,
+        nnipc.protocolType);
+
+    Endpoint ipc2 = findEndpoint(record, IPC2, false, 1,2);
+
+    Endpoint web = findEndpoint(record, "web", true, 1, 1);
+    assertEquals(1, web.addresses.size());
+    assertEquals(1, web.addresses.get(0).size());
+  }
+
+  /**
+   * Assert that an endpoint matches the criteria
+   * @param endpoint endpoint to examine
+   * @param addressType expected address type
+   * @param protocolType expected protocol type
+   * @param api API
+   */
+  public static void assertMatches(Endpoint endpoint,
+      String addressType,
+      String protocolType,
+      String api) {
+    assertNotNull(endpoint);
+    assertEquals(addressType, endpoint.addressType);
+    assertEquals(protocolType, endpoint.protocolType);
+    assertEquals(api, endpoint.api);
+  }
+
+  /**
+   * Assert the records match.
+   * @param source record that was written
+   * @param resolved the one that resolved.
+   */
+  public static void assertMatches(ServiceRecord source, ServiceRecord resolved) {
+    assertNotNull("Null source record ", source);
+    assertNotNull("Null resolved record ", resolved);
+    assertEquals(source.description, resolved.description);
+
+    Map<String, String> srcAttrs = source.attributes();
+    Map<String, String> resolvedAttrs = resolved.attributes();
+    String sourceAsString = source.toString();
+    String resolvedAsString = resolved.toString();
+    assertEquals("Wrong count of attrs in \n" + sourceAsString
+                 + "\nfrom\n" + resolvedAsString,
+        srcAttrs.size(),
+        resolvedAttrs.size());
+    for (Map.Entry<String, String> entry : srcAttrs.entrySet()) {
+      String attr = entry.getKey();
+      assertEquals("attribute "+ attr, entry.getValue(), resolved.get(attr));
+    }
+    assertEquals("wrong external endpoint count",
+        source.external.size(), resolved.external.size());
+    assertEquals("wrong external endpoint count",
+        source.internal.size(), resolved.internal.size());
+  }
+
+  /**
+   * Find an endpoint in a record or fail,
+   * @param record record
+   * @param api API
+   * @param external external?
+   * @param addressElements expected # of address elements?
+   * @param addressTupleSize expected size of a type
+   * @return the endpoint.
+   */
+  public static Endpoint findEndpoint(ServiceRecord record,
+      String api, boolean external, int addressElements, int addressTupleSize) {
+    Endpoint epr = external ? record.getExternalEndpoint(api)
+                            : record.getInternalEndpoint(api);
+    if (epr != null) {
+      assertEquals("wrong # of addresses",
+          addressElements, epr.addresses.size());
+      assertEquals("wrong # of elements in an address tuple",
+          addressTupleSize, epr.addresses.get(0).size());
+      return epr;
+    }
+    List<Endpoint> endpoints = external ? record.external : record.internal;
+    StringBuilder builder = new StringBuilder();
+    for (Endpoint endpoint : endpoints) {
+      builder.append("\"").append(endpoint).append("\" ");
+    }
+    fail("Did not find " + api + " in endpoints " + builder);
+    // never reached; here to keep the compiler happy
+    return null;
+  }
+
+  /**
+   * Log a record
+   * @param name record name
+   * @param record details
+   * @throws IOException only if something bizarre goes wrong marshalling
+   * a record.
+   */
+  public static void logRecord(String name, ServiceRecord record) throws
+      IOException {
+    LOG.info(" {} = \n{}\n", name, recordMarshal.toJson(record));
+  }
+
+  /**
+   * Create a service entry with the sample endpoints
+   * @param persistence persistence policy
+   * @return the record
+   * @throws IOException on a failure
+   */
+  public static ServiceRecord buildExampleServiceEntry(String persistence) throws
+      IOException,
+      URISyntaxException {
+    ServiceRecord record = new ServiceRecord();
+    record.set(YarnRegistryAttributes.YARN_ID, "example-0001");
+    record.set(YarnRegistryAttributes.YARN_PERSISTENCE, persistence);
+    addSampleEndpoints(record, "namenode");
+    return record;
+  }
+
+  /**
+   * Add some endpoints
+   * @param entry entry
+   */
+  public static void addSampleEndpoints(ServiceRecord entry, String hostname)
+      throws URISyntaxException {
+    assertNotNull(hostname);
+    entry.addExternalEndpoint(webEndpoint("web",
+        new URI("http", hostname + ":80", "/")));
+    entry.addExternalEndpoint(
+        restEndpoint(API_WEBHDFS,
+            new URI("http", hostname + ":8020", "/")));
+
+    Endpoint endpoint = ipcEndpoint(API_HDFS, true, null);
+    endpoint.addresses.add(tuple(hostname, "8030"));
+    entry.addInternalEndpoint(endpoint);
+    InetSocketAddress localhost = new InetSocketAddress("localhost", 8050);
+    entry.addInternalEndpoint(
+        inetAddrEndpoint(NNIPC, ProtocolTypes.PROTOCOL_THRIFT, "localhost",
+            8050));
+    entry.addInternalEndpoint(
+        RegistryTypeUtils.ipcEndpoint(
+            IPC2,
+            true,
+            RegistryTypeUtils.marshall(localhost)));
+  }
+
+  /**
+   * Describe the stage in the process with a box around it -so as
+   * to highlight it in test logs
+   * @param log log to use
+   * @param text text
+   * @param args logger args
+   */
+  public static void describe(Logger log, String text, Object...args) {
+    log.info("\n=======================================");
+    log.info(text, args);
+    log.info("=======================================\n");
+  }
+
+  /**
+   * log out from a context if non-null ... exceptions are caught and logged
+   * @param login login context
+   * @return null, always
+   */
+  public static LoginContext logout(LoginContext login) {
+    try {
+      if (login != null) {
+        LOG.debug("Logging out login context {}", login.toString());
+        login.logout();
+      }
+    } catch (LoginException e) {
+      LOG.warn("Exception logging out: {}", e, e);
+    }
+    return null;
+  }
+
+  /**
+   * Exec the native <code>ktutil</code> to list the keys
+   * (primarily to verify that the generated keytabs are compatible).
+   * This operation is not executed on windows. On other platforms
+   * it requires <code>ktutil</code> to be installed and on the path
+   * <pre>
+   *   ktutil --keytab=target/kdc/zookeeper.keytab list --keys
+   * </pre>
+   * @param keytab keytab to list
+   * @throws IOException on any execution problem, including the executable
+   * being missing
+   */
+  public static String ktList(File keytab) throws IOException {
+    if (!Shell.WINDOWS) {
+      String path = keytab.getAbsolutePath();
+      String out = Shell.execCommand(
+          KTUTIL,
+          "--keytab=" + path,
+          "list",
+          "--keys"
+      );
+      LOG.info("Listing of keytab {}:\n{}\n", path, out);
+      return out;
+    }
+    return "";
+  }
+
+  /**
+   * Perform a robust <code>ktutils -l</code> ... catches and ignores
+   * exceptions, otherwise the output is logged.
+   * @param keytab keytab to list
+   * @return the result of the operation, or "" on any problem
+   */
+  public static String ktListRobust(File keytab) {
+    try {
+      return ktList(keytab);
+    } catch (IOException e) {
+      // probably not on the path
+      return "";
+    }
+  }
+
+  /**
+   * Login via a UGI. Requres UGI to have been set up
+   * @param user username
+   * @param keytab keytab to list
+   * @return the UGI
+   * @throws IOException
+   */
+  public static UserGroupInformation loginUGI(String user, File keytab) throws
+      IOException {
+    LOG.info("Logging in as {} from {}", user, keytab);
+    return UserGroupInformation.loginUserFromKeytabAndReturnUGI(user,
+        keytab.getAbsolutePath());
+  }
+
+  public static ServiceRecord createRecord(String persistence) {
+    return createRecord("01", persistence, "description");
+  }
+
+  public static ServiceRecord createRecord(String id, String persistence,
+      String description) {
+    ServiceRecord serviceRecord = new ServiceRecord();
+    serviceRecord.set(YarnRegistryAttributes.YARN_ID, id);
+    serviceRecord.description = description;
+    serviceRecord.set(YarnRegistryAttributes.YARN_PERSISTENCE, persistence);
+    return serviceRecord;
+  }
+
+  public static ServiceRecord createRecord(String id, String persistence,
+      String description, String data) {
+    return createRecord(id, persistence, description);
+  }
+}