diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 76448cf..1089508 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -72,6 +72,7 @@
3.4.66.0.41
+ 2.4.1
@@ -221,6 +222,11 @@
org.apache.hadoop
+ hadoop-yarn-registry
+ ${project.version}
+
+
+ org.apache.hadoophadoop-yarn-server-nodemanager${project.version}
@@ -828,6 +834,24 @@
2.9.1
+
+ org.apache.curator
+ curator-framework
+ ${curator.version}
+
+
+
+ org.apache.curator
+ curator-test
+ ${curator.version}
+
+
+
+ org.skyscreamer
+ jsonassert
+ 1.2.0
+
+
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
index d91c67b..a9b6952 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
@@ -97,6 +97,11 @@
org.apache.hadoophadoop-yarn-client
+
+
+ org.apache.hadoop
+ hadoop-yarn-registry
+ org.apache.hadoop
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
index 2451030..da7433f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
@@ -57,6 +57,7 @@
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.service.ServiceOperations;
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.Shell;
import org.apache.hadoop.yarn.api.ApplicationConstants;
@@ -93,6 +94,13 @@
import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.registry.client.binding.BindingUtils;
+import org.apache.hadoop.yarn.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.yarn.registry.client.services.RegistryOperationsService;
+import org.apache.hadoop.yarn.registry.client.types.CreateFlags;
+import org.apache.hadoop.yarn.registry.client.types.PersistencePolicies;
+import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.log4j.LogManager;
@@ -270,6 +278,9 @@
private final String linux_bash_command = "bash";
private final String windows_command = "cmd /c";
+ @VisibleForTesting
+ RegistryOperationsService registryOperations;
+
/**
* @param args Command line args
*/
@@ -570,6 +581,63 @@ public void run() throws YarnException, IOException {
RegisterApplicationMasterResponse response = amRMClient
.registerApplicationMaster(appMasterHostname, appMasterRpcPort,
appMasterTrackingUrl);
+
+ // Register with the YARN registry if it is enabled
+ boolean registryEnabled =
+ conf.getBoolean(RegistryConstants.KEY_REGISTRY_ENABLED,
+ RegistryConstants.DEFAULT_REGISTRY_ENABLED);
+ if (registryEnabled) {
+ LOG.info("Registering Service");
+ registryOperations = new RegistryOperationsService();
+ registryOperations.init(conf);
+ registryOperations.start();
+ ServiceRecord serviceRecord = new ServiceRecord();
+ String attemptID = this.appAttemptID.toString();
+ String appId = this.appAttemptID.getApplicationId().toString();
+
+ serviceRecord.id = attemptID;
+ serviceRecord.persistence = PersistencePolicies.APPLICATION_ATTEMPT;
+ serviceRecord.description = "Distributed Shell";
+ // if this service offered external RPC/Web access, they
+ // can be added to the service record
+
+ String username = BindingUtils.currentUser();
+ String serviceClass = DSConstants.SERVICE_CLASS_DISTRIBUTED_SHELL;
+ String serviceName = RegistryPathUtils.encodeYarnID(appId);
+ String path =
+ BindingUtils.servicePath(username, serviceClass, serviceName);
+ registryOperations.mkdir(RegistryPathUtils.parentOf(path), true);
+ // app attempt entry
+ registryOperations.create(path + "-attempt", serviceRecord,
+ CreateFlags.OVERWRITE);
+ LOG.info("Registered " + serviceRecord + " at " + path );
+
+ serviceRecord.id = appId;
+ serviceRecord.persistence = PersistencePolicies.APPLICATION;
+ registryOperations.create(path + "-app", serviceRecord,
+ CreateFlags.OVERWRITE);
+
+ // register one that is not deleted
+// serviceRecord.id = appId;
+ serviceRecord.id = "persisting";
+ serviceRecord.persistence = PersistencePolicies.PERMANENT;
+ registryOperations.create(path + "-permanent", serviceRecord,
+ CreateFlags.OVERWRITE);
+
+ // ephemeral entry to show how its lifespan is automatically that
+ // of the registry session, hence no-need to set the ID to match
+ serviceRecord.id = "distributed shell ephemeral";
+
+ serviceRecord.persistence = PersistencePolicies.EPHEMERAL;
+ String path2 = path + "-ephemeral";
+ registryOperations.create(path2, serviceRecord,
+ CreateFlags.EPHEMERAL );
+ LOG.info("Registered ephemeral entry at " + path2);
+
+ }
+
+
+
// Dump out information about cluster capability as seen by the
// resource manager
int maxMem = response.getMaximumResourceCapability().getMemory();
@@ -679,6 +747,8 @@ protected boolean finish() {
}
amRMClient.stop();
+ ServiceOperations.stop(registryOperations);
+ ServiceOperations.stop(timelineClient);
return success;
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
index a86b521..2a9ce72 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
@@ -212,8 +212,8 @@ public static void main(String[] args) {
*/
public Client(Configuration conf) throws Exception {
this(
- "org.apache.hadoop.yarn.applications.distributedshell.ApplicationMaster",
- conf);
+ "org.apache.hadoop.yarn.applications.distributedshell.ApplicationMaster",
+ conf);
}
Client(String appMasterMainClass, Configuration conf) {
@@ -654,7 +654,7 @@ public boolean run() throws IOException, YarnException {
* @throws YarnException
* @throws IOException
*/
- private boolean monitorApplication(ApplicationId appId)
+ protected boolean monitorApplication(ApplicationId appId)
throws YarnException, IOException {
while (true) {
@@ -704,7 +704,7 @@ else if (YarnApplicationState.KILLED == state
return false;
}
- if (System.currentTimeMillis() > (clientStartTime + clientTimeout)) {
+ if (timedOut()) {
LOG.info("Reached client specified timeout for application. Killing application");
forceKillApplication(appId);
return false;
@@ -714,12 +714,46 @@ else if (YarnApplicationState.KILLED == state
}
/**
+ * Get the YARN client
+ * @return the client
+ */
+ public YarnClient getYarnClient() {
+ return yarnClient;
+ }
+
+ /**
+ * Get the client's configuration
+ * @return the configuration
+ */
+ public Configuration getConf() {
+ return conf;
+ }
+
+ public long getClientTimeout() {
+ return clientTimeout;
+ }
+
+ public void setClientTimeout(long clientTimeout) {
+ this.clientTimeout = clientTimeout;
+ }
+
+ /**
+ * Query the clock and timeout settings to decide
+ * whether or not the current run has timed ut
+ * @return true if the client's monitorApplication() operation
+ * has taken too long.
+ */
+ protected boolean timedOut() {
+ return System.currentTimeMillis() > (clientStartTime + clientTimeout);
+ }
+
+ /**
* Kill a submitted application by sending a call to the ASM
* @param appId Application Id to be killed.
* @throws YarnException
* @throws IOException
*/
- private void forceKillApplication(ApplicationId appId)
+ protected void forceKillApplication(ApplicationId appId)
throws YarnException, IOException {
// TODO clarify whether multiple jobs with the same app id can be submitted and be running at
// the same time.
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DSConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DSConstants.java
index 5912f14..67f9a77 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DSConstants.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DSConstants.java
@@ -44,4 +44,10 @@
* Used to validate the local resource.
*/
public static final String DISTRIBUTEDSHELLSCRIPTLEN = "DISTRIBUTEDSHELLSCRIPTLEN";
+
+ /**
+ * Service class used when registering the service
+ */
+ public static final String SERVICE_CLASS_DISTRIBUTED_SHELL =
+ "org-apache-hadoop-distributedshell";
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index d7a1745..cde803e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -32,6 +32,15 @@
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.service.ServiceOperations;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.registry.client.binding.BindingUtils;
+import org.apache.hadoop.yarn.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.yarn.registry.client.services.RegistryOperationsService;
+import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
import org.junit.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -75,8 +84,9 @@ public void setup() throws Exception {
conf.set("yarn.log.dir", "target");
conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
if (yarnCluster == null) {
+ // create a minicluster with the registry enabled
yarnCluster = new MiniYARNCluster(
- TestDistributedShell.class.getSimpleName(), 1, 1, 1, 1, true);
+ TestDistributedShell.class.getSimpleName(), 1, 1, 1, 1, true, true);
yarnCluster.init(conf);
yarnCluster.start();
NodeManager nm = yarnCluster.getNodeManager(0);
@@ -770,5 +780,255 @@ private int verifyContainerLog(int containerNum,
}
return numOfWords;
}
+
+ @Test(timeout = 90000)
+ public void testRegistryOperations() throws Exception {
+
+ // create a client config with an aggressive timeout policy
+ Configuration clientConf = new Configuration(yarnCluster.getConfig());
+ clientConf.setInt(RegistryConstants.KEY_REGISTRY_ZK_CONNECTION_TIMEOUT, 1000);
+ clientConf.setInt(RegistryConstants.KEY_REGISTRY_ZK_RETRY_TIMES, 1);
+ clientConf.setInt(RegistryConstants.KEY_REGISTRY_ZK_RETRY_CEILING, 1);
+ clientConf.setInt(RegistryConstants.KEY_REGISTRY_ZK_RETRY_INTERVAL, 500);
+ clientConf.setInt(RegistryConstants.KEY_REGISTRY_ZK_SESSION_TIMEOUT, 2000);
+
+ // create a registry operations instance
+ RegistryOperationsService regOps = new RegistryOperationsService();
+ regOps.init(clientConf);
+ regOps.start();
+ LOG.info("Registry Binding: " + regOps);
+
+ // do a simple registry operation to verify that it is live
+ regOps.listDir("/");
+
+ try {
+ String[] args = {
+ "--jar",
+ APPMASTER_JAR,
+ "--num_containers",
+ "1",
+ "--shell_command",
+ "sleep 15",
+ "--master_memory",
+ "512",
+ "--container_memory",
+ "128",
+ };
+
+ LOG.info("Initializing DS Client");
+ RegistryMonitoringClient client =
+ new RegistryMonitoringClient(clientConf);
+
+ client.init(args);
+ LOG.info("Running DS Client");
+ boolean result;
+ try {
+ result = client.run();
+ } finally {
+ client.stop();
+ }
+
+ LOG.info("Client run completed. Result=" + result);
+
+ // application should have found service records
+ ServiceRecord serviceRecord = client.appAttemptRecord;
+ LOG.info("Service record = " + serviceRecord);
+ IOException lookupException =
+ client.lookupException;
+ if (serviceRecord == null && lookupException != null) {
+ LOG.error("Lookup of " + client.servicePath
+ + " failed with " + lookupException, lookupException);
+ throw lookupException;
+ }
+
+ // the app should have succeeded or returned a failure message
+ if (!result) {
+ Assert.fail("run returned false: " + client.failureText);
+ }
+
+ // the app-level record must have been retrieved
+ Assert.assertNotNull("No application record at " + client.appRecordPath,
+ client.appRecord);
+
+ // sleep to let some async operations in the RM continue
+ Thread.sleep(10000);
+ // after the app finishes its records should have been purged
+ assertDeleted(regOps, client.appRecordPath);
+ assertDeleted(regOps, client.servicePath);
+ } finally {
+ regOps.stop();
+ }
+ }
+
+ protected void assertDeleted(RegistryOperationsService regOps,
+ String path) throws IOException {
+ try {
+ ServiceRecord record = regOps.resolve(path);
+ Assert.fail("Expected the record at " + path + " to have been purged,"
+ + " but found " + record);
+ } catch (PathNotFoundException expected) {
+ // expected
+ }
+ }
+
+
+ /**
+ * This is a subclass of the distributed shell client which
+ * monitors the registry as well as the YARN app status
+ */
+ private class RegistryMonitoringClient extends Client {
+ private String servicePath;
+ private ServiceRecord permanentRecord;
+ private String permanentPath;
+ private IOException lookupException;
+ private ServiceRecord appAttemptRecord;
+ private String appAttemptPath;
+
+ private ServiceRecord ephemeralRecord;
+ private String ephemeralPath;
+
+ private ServiceRecord appRecord;
+ private String appRecordPath;
+
+
+ private String failureText;
+ private ApplicationReport report;
+ private final RegistryOperationsService regOps;
+
+ private RegistryMonitoringClient(Configuration conf) throws Exception {
+ super(conf);
+ // client timeout of 30s for the test runs
+ setClientTimeout(30000);
+ regOps = new RegistryOperationsService();
+ regOps.init(getConf());
+ regOps.start();
+ }
+
+ public void stop() {
+ ServiceOperations.stopQuietly(regOps);
+ }
+
+
+ @Override
+ protected boolean monitorApplication(ApplicationId appId)
+ throws YarnException, IOException {
+
+ String username = BindingUtils.currentUser();
+ String serviceClass = DSConstants.SERVICE_CLASS_DISTRIBUTED_SHELL;
+ String serviceName = RegistryPathUtils.encodeYarnID(appId.toString());
+ servicePath =
+ BindingUtils.servicePath(username, serviceClass, serviceName);
+ appAttemptPath = servicePath + "-attempt";
+ ephemeralPath = servicePath + "-ephemeral";
+ appRecordPath = servicePath + "-app";
+ permanentPath = servicePath + "-permanent";
+
+ YarnClient yarnClient = getYarnClient();
+
+ while (!timedOut()) {
+
+ // Check app status every 1 second.
+ try {
+ Thread.sleep(500);
+ } catch (InterruptedException e) {
+ LOG.debug("Thread sleep in monitoring loop interrupted");
+ }
+
+ // Get application report for the appId we are interested in
+ report = yarnClient.getApplicationReport(appId);
+
+ YarnApplicationState state =
+ report.getYarnApplicationState();
+ switch (state) {
+
+ case NEW:
+ case NEW_SAVING:
+ case SUBMITTED:
+ case ACCEPTED:
+ continue;
+
+ // running, extract service records if not already done
+ case RUNNING:
+ try {
+ permanentRecord = maybeResolve(permanentRecord, permanentPath);
+ // succesfull lookup, so discard any failure
+ lookupException = null;
+ } catch (PathNotFoundException e) {
+ lookupException = e;
+ }
+ appRecord = maybeResolveQuietly(appRecord, appRecordPath);
+ appAttemptRecord = maybeResolveQuietly(appAttemptRecord,
+ appAttemptPath);
+ ephemeralRecord = maybeResolveQuietly(ephemeralRecord,
+ ephemeralPath);
+ continue;
+
+ case FINISHED:
+ // completed
+ boolean read = permanentRecord != null;
+ if (!read) {
+ failureText = "Permanent record was not resolved";
+ }
+ return read;
+
+ case KILLED:
+ failureText = "Application Killed: " + report.getDiagnostics();
+ return false;
+
+ case FAILED:
+ failureText = "Application Failed: " + report.getDiagnostics();
+ return false;
+
+ default:
+ break;
+ }
+
+ }
+
+ if (timedOut()) {
+ failureText = "Timed out: Killing application";
+ forceKillApplication(appId);
+ }
+ return false;
+ }
+
+ /**
+ * Resolve a record if it has not been resolved already
+ * @param r record
+ * @param path path
+ * @return r if it was non null, else the resolved record
+ * @throws IOException on any failure
+ */
+ ServiceRecord maybeResolve(ServiceRecord r, String path) throws IOException {
+ if (r == null) {
+ ServiceRecord record = regOps.resolve(path);
+ LOG.info("Resolved at " + r +": " + record);
+ return record;
+ }
+ return r;
+ }
+
+ /**
+ * Resolve a record if it has not been resolved already —ignoring
+ * any PathNotFoundException exceptions.
+ * @param r record
+ * @param path path
+ * @return r if it was non null, a resolved record if it was found,
+ * or null if the resolution failed with a PathNotFoundException
+ * @throws IOException on any failure
+ */
+ ServiceRecord maybeResolveQuietly(ServiceRecord r, String path) throws
+ IOException {
+ try {
+ return maybeResolve(r, path);
+ } catch (PathNotFoundException ignored) {
+ // ignored
+ }
+ return r;
+ }
+
+ } // end of class RegistryMonitoringClient
+
+
}
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..6a1603f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml
@@ -0,0 +1,239 @@
+
+
+
+
+ hadoop-yarn
+ org.apache.hadoop
+ 3.0.0-SNAPSHOT
+
+ 4.0.0
+ org.apache.hadoop
+ hadoop-yarn-registry
+ 3.0.0-SNAPSHOT
+ hadoop-yarn-registry
+
+
+
+ ${project.parent.basedir}
+
+
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-api
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-common
+
+
+
+ org.apache.hadoop
+ hadoop-common
+
+
+
+
+ org.apache.hadoop
+ hadoop-common
+ test-jar
+ test
+
+
+
+ com.sun.jersey
+ jersey-core
+
+
+
+ commons-logging
+ commons-logging
+
+
+
+ org.codehaus.jackson
+ jackson-core-asl
+ compile
+
+
+
+ org.codehaus.jackson
+ jackson-mapper-asl
+ compile
+
+
+
+ commons-httpclient
+ commons-httpclient
+ test
+
+
+
+ junit
+ junit
+ test
+
+
+
+ org.skyscreamer
+ jsonassert
+ test
+
+
+
+ org.apache.curator
+ curator-framework
+
+
+
+ org.apache.curator
+ curator-test
+ test
+
+
+
+
+
+
+
+
+
+ ${basedir}/src/main/resources
+
+ yarn-version-info.properties
+
+ false
+
+
+ ${basedir}/src/main/resources
+
+ yarn-version-info.properties
+
+ true
+
+
+
+
+ org.apache.rat
+ apache-rat-plugin
+
+
+
+
+
+
+ org.apache.hadoop
+ hadoop-maven-plugins
+
+
+ version-info
+ generate-resources
+
+ version-info
+
+
+
+ ${basedir}/src/main
+
+ java/**/*.java
+
+
+
+
+
+
+
+
+ maven-jar-plugin
+
+
+
+ test-jar
+
+ test-compile
+
+
+
+
+
+
+
+
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/api/RegistryConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/api/RegistryConstants.java
new file mode 100644
index 0000000..a6141f0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/api/RegistryConstants.java
@@ -0,0 +1,173 @@
+/*
+ * 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.yarn.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}
+ */
+ String REGISTRY_PREFIX = "yarn.registry";
+
+ /**
+ * flag to indicate whether or not the registry should
+ * be enabled: {@value}
+ */
+ String KEY_REGISTRY_ENABLED = REGISTRY_PREFIX + ".enabled";
+
+ boolean DEFAULT_REGISTRY_ENABLED = false;
+
+ /**
+ * Flag to indicate whether the ZK service should be enabled {@value}
+ * in the RM
+ */
+ String KEY_ZKSERVICE_ENABLED = REGISTRY_PREFIX + ".zkservice.enabled";
+ boolean DEFAULT_ZKSERVICE_ENABLED = false;
+
+
+ /**
+ * List of hostname:port pairs defining the ZK quorum: {@value}
+ */
+ String KEY_REGISTRY_ZK_QUORUM = REGISTRY_PREFIX + ".zk.quorum";
+
+ /**
+ * Zookeeper session timeout in milliseconds: {@value}
+ */
+ String KEY_REGISTRY_ZK_SESSION_TIMEOUT =
+ REGISTRY_PREFIX + ".zk.session-timeout-ms";
+
+ /**
+ * Zookeeper connect retry count: {@value}
+ */
+ String KEY_REGISTRY_ZK_RETRY_TIMES = REGISTRY_PREFIX + ".zk.retry.times";
+
+ /**
+ * Zookeeper connection timeout in milliseconds: {@value}
+ */
+
+ String KEY_REGISTRY_ZK_CONNECTION_TIMEOUT =
+ REGISTRY_PREFIX + ".zk.connection-timeout-ms";
+
+ /**
+ * Zookeeper connect interval in milliseconds: {@value}
+ */
+ String KEY_REGISTRY_ZK_RETRY_INTERVAL =
+ REGISTRY_PREFIX + ".zk.retry.interval-ms";
+
+ /**
+ * Zookeeper retry limit in milliseconds: {@value}
+ */
+ String KEY_REGISTRY_ZK_RETRY_CEILING =
+ REGISTRY_PREFIX + ".zk.retry.ceiling-ms";
+
+ /**
+ * Root path in the ZK tree for the registry: {@value}
+ */
+ String KEY_REGISTRY_ZK_ROOT = REGISTRY_PREFIX + ".zk.root";
+
+ /**
+ * Root path in the ZK tree for the registry: {@value}
+ */
+ String KEY_REGISTRY_ZK_ACL = REGISTRY_PREFIX + ".zk.acl";
+
+ /**
+ * The default ZK quorum binding: {@value}
+ */
+ String DEFAULT_ZK_HOSTS = "localhost:2181";
+
+ /**
+ * The default permissions for the registry root: {@value}
+ */
+ String DEFAULT_REGISTRY_ROOT_PERMISSIONS = "world:anyone:rwcda";
+
+ /**
+ * The default ZK session timeout: {@value}
+ */
+ int DEFAULT_ZK_SESSION_TIMEOUT = 20000;
+ /**
+ * The default ZK session timeout: {@value}
+ */
+ int DEFAULT_ZK_CONNECTION_TIMEOUT = 15000;
+ /**
+ * The default # of times to retry a ZK connection: {@value}
+ */
+ int DEFAULT_ZK_RETRY_TIMES = 5;
+ /**
+ * The default interval between connection retries: {@value}
+ */
+ int DEFAULT_ZK_RETRY_INTERVAL = 1000;
+ /**
+ * Default limit on retries: {@value}
+ */
+ int DEFAULT_ZK_RETRY_CEILING = 5;
+ String DEFAULT_REGISTRY_ROOT = "/yarnRegistry";
+
+ /**
+ * Pattern of a hostname : {@value}
+ */
+ String HOSTNAME_PATTERN =
+ "([a-z0-9]|[a-z0-9][a-z0-9\\-]*[a-z0-9])";
+ /**
+ * 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 under a service record to point to components of that service:
+ * {@value}
+ */
+ String SUBPATH_COMPONENTS = "/components";
+
+ /**
+ * Header of a service record: {@value}
+ * 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
+ */
+ byte[] RECORD_HEADER = {'j', 's', 'o', 'n',
+ 's', 'e','r','v','i', 'c', 'e',
+ 'r','e','c'};
+
+ /**
+ * ZK servertick time: {@value}
+ */
+ String KEY_ZKSERVICE_TICK_TIME = REGISTRY_PREFIX + ".zkservice.ticktime";
+
+ /**
+ * port; 0 or below means "any": {@value}
+ */
+ String KEY_ZKSERVICE_PORT = REGISTRY_PREFIX + ".zkservice.port";
+
+ /**
+ * Directory containing data: {@value}
+ */
+ String KEY_ZKSERVICE_DATADIR = REGISTRY_PREFIX + ".zkservice.datadir";
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/api/RegistryOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/api/RegistryOperations.java
new file mode 100644
index 0000000..528b53d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/api/RegistryOperations.java
@@ -0,0 +1,158 @@
+/*
+ * 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.yarn.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.security.AccessControlException;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.registry.client.exceptions.InvalidPathnameException;
+import org.apache.hadoop.yarn.registry.client.exceptions.NoChildrenForEphemeralsException;
+import org.apache.hadoop.yarn.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
+
+import java.io.IOException;
+
+/**
+ * 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 NoChildrenForEphemeralsException the parent is ephemeral.
+ * @throws AccessControlException access permission failure.
+ * @throws InvalidPathnameException path name is invalid.
+ * @throws IOException Any other IO Exception.
+ * @return true if the path was created, false if it existed.
+ */
+ boolean mkdir(String path, boolean createParents)
+ throws PathNotFoundException,
+ NoChildrenForEphemeralsException,
+ AccessControlException,
+ 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 NoChildrenForEphemeralsException the parent is ephemeral
+ * @throws FileAlreadyExistsException path exists but create flags
+ * do not include "overwrite"
+ * @throws AccessControlException access permission failure.
+ * @throws InvalidPathnameException path name is invalid.
+ * @throws IOException Any other IO Exception.
+ */
+ void create(String path, ServiceRecord record, int createFlags)
+ throws PathNotFoundException,
+ NoChildrenForEphemeralsException,
+ FileAlreadyExistsException,
+ AccessControlException,
+ 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 AccessControlException security restriction.
+ * @throws InvalidPathnameException the path is invalid.
+ * @throws IOException Any other IO Exception
+ */
+
+ ServiceRecord resolve(String path) throws PathNotFoundException,
+ AccessControlException,
+ InvalidPathnameException,
+ IOException;
+
+ /**
+ * Get the status of a path
+ * @param path
+ * @return
+ * @throws PathNotFoundException path is not in the registry.
+ * @throws AccessControlException security restriction.
+ * @throws InvalidPathnameException the path is invalid.
+ * @throws IOException Any other IO Exception
+ */
+ RegistryPathStatus stat(String path)
+ throws PathNotFoundException,
+ AccessControlException,
+ InvalidPathnameException,
+ IOException;
+
+ /**
+ * List children of a directory
+ * @param path path
+ * @return a possibly empty array of child entries
+ * @throws PathNotFoundException path is not in the registry.
+ * @throws AccessControlException security restriction.
+ * @throws InvalidPathnameException the path is invalid.
+ * @throws IOException Any other IO Exception
+ */
+ RegistryPathStatus[] listDir(String path)
+ throws PathNotFoundException,
+ AccessControlException,
+ 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 AccessControlException security restriction.
+ * @throws InvalidPathnameException the path is invalid.
+ * @throws PathIsNotEmptyDirectoryException path has child entries, but
+ * recursive is false.
+ * @throws IOException Any other IO Exception
+ *
+ * @throws IOException
+ */
+ void delete(String path, boolean recursive)
+ throws PathNotFoundException,
+ PathIsNotEmptyDirectoryException,
+ AccessControlException,
+ InvalidPathnameException,
+ IOException;
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/BindingUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/BindingUtils.java
new file mode 100644
index 0000000..e4ac8e0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/BindingUtils.java
@@ -0,0 +1,125 @@
+/*
+ * 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.yarn.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.security.UserGroupInformation;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.yarn.registry.client.api.RegistryConstants.*;
+
+/**
+ * Methods for binding paths according to recommended layout, and for
+ * extracting some of the content
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class BindingUtils {
+
+ /**
+ * Buld the user path -switches to the system path if the user is "".
+ * It also cross-converts the username to ascii via punycode
+ * @param user username or ""
+ * @return the path to the user
+ */
+ public static String userPath(String user) {
+ Preconditions.checkArgument(user != null, "null user");
+ if (user.isEmpty()) {
+ return PATH_SYSTEM_SERVICES;
+ }
+
+ return RegistryPathUtils.join(PATH_USERS,
+ RegistryPathUtils.encodeForRegistry(user));
+ }
+
+ /**
+ * Create a service classpath
+ * @param user username or ""
+ * @param serviceClass service name
+ * @return a full path
+ */
+ public static String serviceclassPath(String user,
+ String serviceClass) {
+
+ return RegistryPathUtils.join(userPath(user),
+ serviceClass);
+ }
+
+
+ /**
+ * Get the current user path formatted for the system
+ * @return
+ * @throws IOException
+ */
+ public static String currentUser() throws IOException {
+ return RegistryPathUtils.encodeForRegistry(
+ UserGroupInformation.getCurrentUser().getShortUserName());
+ }
+
+ /**
+ * 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 RegistryPathUtils.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 RegistryPathUtils.join(servicePath(user, serviceClass, serviceName),
+ 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 RegistryPathUtils.join(
+ componentListPath(user, serviceClass, serviceName),
+ componentName);
+ }
+
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/JsonSerDeser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/JsonSerDeser.java
new file mode 100644
index 0000000..a5d99c7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/JsonSerDeser.java
@@ -0,0 +1,319 @@
+/*
+ * 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.yarn.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.yarn.registry.client.exceptions.InvalidRecordException;
+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.
+ * It constructs an object mapper as an instance field.
+ * and synchronizes access to those methods
+ * which use the mapper
+ * @param
+ */
+@InterfaceAudience.Private()
+@InterfaceStability.Evolving
+public class JsonSerDeser {
+
+ private static final Logger LOG = LoggerFactory.getLogger(JsonSerDeser.class);
+ private static final String UTF_8 = "UTF-8";
+
+ private final Class 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 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);
+ this.header = header;
+ }
+
+ 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 (T) (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 (T) (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
+ */
+ 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 (T) (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
+ * @throws IOException on a failure
+ */
+ @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 InvalidRecordException(path,
+ "Record too short for header of " + getName());
+ }
+ byte[] magic = Arrays.copyOfRange(buffer, 0, hlen);
+ if (!Arrays.equals(header, magic)) {
+ throw new InvalidRecordException(path,
+ "Entry header does not match header of " + getName());
+ }
+ }
+ 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);
+ }
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/RecordOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/RecordOperations.java
new file mode 100644
index 0000000..22e891c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/RecordOperations.java
@@ -0,0 +1,82 @@
+/*
+ * 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.yarn.registry.client.binding;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.registry.client.api.RegistryOperations;
+import org.apache.hadoop.yarn.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.yarn.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Support for operations on records
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RecordOperations {
+ private static final Logger LOG = LoggerFactory.getLogger(JsonSerDeser.class);
+
+ public static class ServiceRecordMarshal extends JsonSerDeser {
+ public ServiceRecordMarshal() {
+ super(ServiceRecord.class, RegistryConstants.RECORD_HEADER);
+ }
+ }
+
+ /**
+ * 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 stats list of stat results
+ * @return a possibly empty list
+ * @throws IOException for any IO Operation that wasn't ignored.
+ */
+ public static Map extractServiceRecords(RegistryOperations operations,
+ RegistryPathStatus[] stats) throws IOException {
+ Map results = new HashMap(stats.length);
+ for (RegistryPathStatus stat : stats) {
+ if (stat.size > RegistryConstants.RECORD_HEADER.length) {
+ // maybe has data
+ try {
+ ServiceRecord serviceRecord = operations.resolve(stat.path);
+ results.put(stat.path, serviceRecord);
+ } catch (EOFException ignored) {
+ LOG.debug("data too short for {}", stat.path);
+ } catch (InvalidRecordException record) {
+ LOG.debug("Invalid record at {}", stat.path);
+ }
+
+ }
+ }
+ return results;
+
+ }
+
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/RegistryPathUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/RegistryPathUtils.java
new file mode 100644
index 0000000..a293027
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/RegistryPathUtils.java
@@ -0,0 +1,187 @@
+/*
+ * 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.yarn.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.yarn.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.registry.client.exceptions.InvalidPathnameException;
+import org.apache.zookeeper.common.PathUtils;
+
+import java.io.IOException;
+import java.net.IDN;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Pattern;
+
+/**
+ * Low level zookeeper-related operations on paths
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RegistryPathUtils {
+
+
+ private static final Pattern HOSTNAME =
+ Pattern.compile(RegistryConstants.HOSTNAME_PATTERN);
+
+ /**
+ * Validate ZK path with the path itself included in
+ * the exception text
+ * @param path path to validate
+ */
+ 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 with the path itself included in
+ * the exception text
+ * @param path path to validate
+ */
+ public static String validateElementsAsDNS(String path) throws
+ InvalidPathnameException {
+ List splitpath = split(path);
+ for (String fragment : splitpath) {
+ if (!HOSTNAME.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 IllegalArgumentException if the path is invalide
+ */
+ public static String createFullPath(String base, String path) throws
+ IOException {
+ 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);
+
+ if (!path.startsWith("/") && !fullpath.toString().endsWith("/")) {
+ fullpath.append("/");
+ }
+ 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 split(String path) {
+ //
+ String[] pathelements = path.split("/");
+ List dirs = new ArrayList(pathelements.length);
+ for (String pathelement : pathelements) {
+ if (!pathelement.isEmpty()) {
+ dirs.add(pathelement);
+ }
+ }
+ return dirs;
+ }
+
+ /**
+ * Get the parent of a path
+ * @param path
+ * @return the parent path
+ * @throws PathNotFoundException if the path was at root.
+ */
+ public static String parentOf(String path) throws PathNotFoundException {
+ List 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("_", "-");
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/RegistryTypeUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/RegistryTypeUtils.java
new file mode 100644
index 0000000..149ad31
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/RegistryTypeUtils.java
@@ -0,0 +1,170 @@
+/*
+ * 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.yarn.registry.client.binding;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.yarn.registry.client.types.AddressTypes;
+import org.apache.hadoop.yarn.registry.client.types.Endpoint;
+import org.apache.hadoop.yarn.registry.client.types.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;
+
+/**
+ * Utils to work with registry types
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RegistryTypeUtils {
+
+ public static Endpoint urlEndpoint(String api,
+ String protocolType,
+ URI... urls) {
+ return new Endpoint(api, protocolType, urls);
+ }
+
+ public static Endpoint restEndpoint(String api,
+ URI... urls) {
+ return urlEndpoint(api, ProtocolTypes.PROTOCOL_REST, urls);
+ }
+
+ public static Endpoint webEndpoint(String api,
+ URI... urls) {
+ return urlEndpoint(api, ProtocolTypes.PROTOCOL_WEBUI, urls);
+ }
+
+ public static Endpoint inetAddrEndpoint(String api,
+ String protocolType,
+ String hostname,
+ int port) {
+ return new Endpoint(api,
+ AddressTypes.ADDRESS_HOSTNAME_AND_PORT,
+ protocolType,
+ RegistryTypeUtils.tuple(hostname, Integer.toString(port)));
+ }
+
+ public static Endpoint ipcEndpoint(String api,
+ boolean protobuf, List address) {
+ return new Endpoint(api,
+ AddressTypes.ADDRESS_HOSTNAME_AND_PORT,
+ protobuf ? ProtocolTypes.PROTOCOL_HADOOP_IPC_PROTOBUF
+ : ProtocolTypes.PROTOCOL_HADOOP_IPC,
+ address
+ );
+ }
+
+ public static List tuple(String... t1) {
+ return Arrays.asList(t1);
+ }
+
+ public static List tuple(Object... t1) {
+ List l = new ArrayList(t1.length);
+ for (Object t : t1) {
+ l.add(t.toString());
+ }
+ return l;
+ }
+
+ /**
+ * Convert a socket address pair into a string tuple, (host, port)
+ * @param address an address
+ * @return an element for the address list
+ */
+ public static List marshall(InetSocketAddress address) {
+ return tuple(address.getHostString(), address.getPort());
+ }
+
+ /**
+ * Require a specific address type on an endpoint
+ * @param required required type
+ * @param epr endpoint
+ * @throws IllegalStateException 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 retrieveAddressesUriType(Endpoint epr) throws
+ InvalidRecordException {
+ if (epr == null) {
+ return null;
+ }
+ requireAddressType(AddressTypes.ADDRESS_URI, epr);
+ List> addresses = epr.addresses;
+ if (addresses.size() < 1) {
+ throw new InvalidRecordException(epr.toString(),
+ "No addresses in endpoint");
+ }
+ List results = new ArrayList(addresses.size());
+ for (List address : addresses) {
+ if (address.size() != 1) {
+ throw new InvalidRecordException(epr.toString(),
+ "Address payload invalid: wrong many 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 retrieveAddressURLs(Endpoint epr) throws
+ InvalidRecordException,
+ MalformedURLException {
+ if (epr == null) {
+ throw new InvalidRecordException("", "Null endpoint");
+ }
+ List addresses = retrieveAddressesUriType(epr);
+ List results = new ArrayList(addresses.size());
+ for (String address : addresses) {
+ results.add(new URL(address));
+ }
+ return results;
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/ZKPathDumper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/ZKPathDumper.java
new file mode 100644
index 0000000..7370028
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/binding/ZKPathDumper.java
@@ -0,0 +1,110 @@
+/*
+ * 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.yarn.registry.client.binding;
+
+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.Stat;
+
+import java.util.List;
+
+/**
+ * This class dumps a registry tree to a string.
+ * It does this in the toString() 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;
+
+ /**
+ * Create a path dumper -but do not dump the path until asked
+ * @param curator curator instance
+ * @param root root
+ */
+ public ZKPathDumper(CuratorFramework curator,
+ String root) {
+ Preconditions.checkArgument(curator != null);
+ Preconditions.checkArgument(root != null);
+ this.curator = curator;
+ this.root = root;
+ }
+
+ @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 children = childrenBuilder.forPath(path);
+ for (String child : children) {
+ String childPath = path + "/" + child;
+ String body = "";
+ Stat stat = curator.checkExists().forPath(childPath);
+ StringBuilder verboseDataBuilder = new StringBuilder(64);
+ verboseDataBuilder.append(" [")
+ .append(stat.getDataLength())
+ .append("]");
+ if (stat.getEphemeralOwner() > 0) {
+ verboseDataBuilder.append("*");
+ }
+ body = verboseDataBuilder.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");
+ }
+
+ }
+
+ private void append(StringBuilder builder, int indent, char c) {
+ for (int i = 0; i < indent; i++) {
+ builder.append(c);
+ }
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/exceptions/InvalidPathnameException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/exceptions/InvalidPathnameException.java
new file mode 100644
index 0000000..40809f7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/exceptions/InvalidPathnameException.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.yarn.registry.client.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A path name was invalid
+ */
+@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);
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/exceptions/InvalidRecordException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/exceptions/InvalidRecordException.java
new file mode 100644
index 0000000..0ee2b9d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/exceptions/InvalidRecordException.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.yarn.registry.client.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Raised if the entry doesn't match what was expected at the path.
+ */
+@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);
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/exceptions/NoChildrenForEphemeralsException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/exceptions/NoChildrenForEphemeralsException.java
new file mode 100644
index 0000000..70df187
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/exceptions/NoChildrenForEphemeralsException.java
@@ -0,0 +1,43 @@
+/*
+ * 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.yarn.registry.client.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Children are not allowed under ephemeral nodes
+ */
+@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);
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/exceptions/RegistryIOException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/exceptions/RegistryIOException.java
new file mode 100644
index 0000000..a82ac93
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/exceptions/RegistryIOException.java
@@ -0,0 +1,43 @@
+/*
+ * 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.yarn.registry.client.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.PathIOException;
+
+/**
+ * Exception for registry operations
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RegistryIOException extends PathIOException {
+
+ 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);
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/services/BindingInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/services/BindingInformation.java
new file mode 100644
index 0000000..0fb9cfb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/services/BindingInformation.java
@@ -0,0 +1,34 @@
+/*
+ * 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.yarn.registry.client.services;
+
+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 {
+
+ public EnsembleProvider ensembleProvider;
+ public String description;
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/services/CuratorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/services/CuratorService.java
new file mode 100644
index 0000000..3aa36d4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/services/CuratorService.java
@@ -0,0 +1,612 @@
+/*
+ * 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.yarn.registry.client.services;
+
+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.PathAccessDeniedException;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.ZKUtil;
+import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.yarn.registry.client.binding.ZKPathDumper;
+import org.apache.hadoop.yarn.registry.client.exceptions.NoChildrenForEphemeralsException;
+import org.apache.hadoop.yarn.registry.client.exceptions.RegistryIOException;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+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 RegistryOperations API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class CuratorService extends AbstractService
+ implements RegistryConstants, RegistryBindingSource {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(CuratorService.class);
+
+ /**
+ * the Curator binding
+ */
+ private CuratorFramework curator;
+
+ /**
+ * Parsed root ACL
+ */
+ private List rootACL;
+
+ /**
+ * Path to the registry root
+ */
+ private String registryRoot;
+
+ private final RegistryBindingSource bindingSource;
+
+ /**
+ * the connection binding text for messages
+ */
+ private String connectionDescription;
+ 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);
+ }
+
+ @Override
+ protected void serviceStart() throws Exception {
+ super.serviceStart();
+
+ registryRoot = getConfig().getTrimmed(KEY_REGISTRY_ZK_ROOT,
+ DEFAULT_REGISTRY_ROOT) ;
+ LOG.debug("Creating Registry with root {}", registryRoot);
+
+ rootACL = getACLs(KEY_REGISTRY_ZK_ACL, DEFAULT_REGISTRY_ROOT_PERMISSIONS);
+ curator = newCurator();
+ }
+
+ /**
+ * Close the ZK connection if it is open
+ */
+ @Override
+ public void serviceStop() throws Exception {
+ IOUtils.closeStream(curator);
+ }
+
+ /**
+ * Get the ACLs defined in the config key for this service, or
+ * the default
+ * @param confKey configuration key
+ * @param defaultPermissions default values
+ * @return an ACL list.
+ * @throws IOException
+ * @throws ZKUtil.BadAclFormatException on a bad ACL parse
+ */
+ public List getACLs(String confKey, String defaultPermissions) throws
+ IOException, ZKUtil.BadAclFormatException {
+ String zkAclConf = getConfig().get(confKey, defaultPermissions);
+ return parseACLs(zkAclConf);
+ }
+
+ /**
+ * Parse an ACL list. This includes configuration indirection
+ * {@link ZKUtil#resolveConfIndirection(String)}
+ * @param zkAclConf configuration string
+ * @return an ACL list
+ * @throws IOException
+ * @throws ZKUtil.BadAclFormatException on a bad ACL parse
+ */
+ public List parseACLs(String zkAclConf) throws IOException,
+ ZKUtil.BadAclFormatException {
+ return ZKUtil.parseACLs(ZKUtil.resolveConfIndirection(zkAclConf));
+ }
+
+ /**
+ * 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 newCurator() {
+ 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);
+
+ LOG.debug("Creating CuratorService with connection {}",
+ connectionDescription);
+ CuratorFrameworkFactory.Builder b = CuratorFrameworkFactory.builder();
+ b.ensembleProvider(ensembleProvider)
+ .connectionTimeoutMs(connectionTimeout)
+ .sessionTimeoutMs(sessionTimeout)
+ .retryPolicy(new BoundedExponentialBackoffRetry(retryInterval,
+ retryTimes,
+ retryCeiling));
+
+
+/*
+ if (!root.isEmpty()) {
+ String namespace = root;
+ if (namespace.startsWith("/")) {
+ namespace = namespace.substring(1);
+ }
+ b.namespace(namespace);
+ }
+*/
+
+ CuratorFramework framework = b.build();
+ framework.start();
+
+ return framework;
+ }
+
+ @Override
+ public String toString() {
+ return super.toString()
+ + bindingDiagnosticDetails();
+
+ }
+
+ public String bindingDiagnosticDetails() {
+ return " ZK quorum=\"" + connectionDescription + "\""
+ + " root=\"" + registryRoot + "\"";
+ }
+
+ /**
+ * 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();
+ String connectString = buildConnectionString();
+ connectionDescription = binding.description;
+ ensembleProvider = binding.ensembleProvider;
+ }
+
+
+ /**
+ * Supply the binding information.
+ * This implementation returns a fixed ensemble bonded to
+ * the quorum supplied by {@link #buildConnectionString()}
+ * @return
+ */
+ @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_ZK_HOSTS);
+ }
+
+ /**
+ * 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) {
+ IOException ioe;
+ 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 PathAccessDeniedException(path);
+ } else if (exception instanceof KeeperException.NotEmptyException) {
+ ioe = new PathIsNotEmptyDirectoryException(path);
+ } else if (exception instanceof KeeperException.NoChildrenForEphemeralsException) {
+ ioe = new NoChildrenForEphemeralsException(path,
+ "Cannot create a path under an ephemeral node:" + exception.toString(),
+ exception);
+ } 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
+ * @return true iff the path was created
+ * @throws IOException
+ */
+ @VisibleForTesting
+ public boolean maybeCreate(String path, CreateMode mode) throws IOException {
+ List acl = rootACL;
+ return maybeCreate(path, mode, acl, false);
+ }
+
+ /**
+ * 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,
+ 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 {
+ 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;
+ }
+
+ /**
+ * Poll for a path existing
+ * @param path path of operation
+ * @return true if the path was visible from the ZK server
+ * queried.
+ * @throws IOException
+ */
+ public boolean zkPathExists(String path) throws IOException {
+ try {
+ return zkStat(path) != null;
+ } catch (PathNotFoundException e) {
+ return false;
+ } catch (Exception e) {
+ throw operationFailure(path, "existence check", 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 path
+ * @param path path to create
+ * @param mode mode for path
+ * @throws IOException
+ */
+ public void zkMkPath(String path, CreateMode mode) throws IOException {
+ zkMkPath(path, mode, false, rootACL);
+ }
+
+ /**
+ * 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 acl ACL for path
+ * @throws IOException any problem
+ */
+ protected boolean zkMkPath(String path,
+ CreateMode mode,
+ boolean createParents,
+ List acl)
+ throws IOException {
+ path = createFullPath(path);
+ try {
+ CreateBuilder createBuilder = curator.create();
+ createBuilder.withMode(mode).withACL(acl);
+ if (createParents) {
+ createBuilder.creatingParentsIfNeeded();
+ }
+ createBuilder.forPath(path);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Created path {} with mode {} and ACL {}", path, mode, acl);
+ }
+ } catch (KeeperException.NodeExistsException e) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("path already present: {}", path, e);
+ }
+ return false;
+ } catch (Exception e) {
+ throw operationFailure(path, "mkdir() ", e);
+ }
+ 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) 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 acl
+ * @throws IOException
+ */
+ public void zkCreate(String path,
+ CreateMode mode,
+ byte[] data,
+ List acl) throws IOException {
+ Preconditions.checkArgument(data != null, "null data");
+ String fullpath = createFullPath(path);
+ try {
+ LOG.debug("Creating {} with {} bytes", fullpath, data.length);
+ curator.create().withMode(mode).withACL(acl).forPath(fullpath, data);
+ } catch (Exception e) {
+ throw operationFailure(fullpath, "create()", e);
+ }
+ }
+
+ /**
+ * 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");
+ path = createFullPath(path);
+ try {
+ 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, boolean overwrite) throws IOException {
+ Preconditions.checkArgument(data != null, "null data");
+ 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 {
+ String fullpath = createFullPath(path);
+ try {
+ 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 zkList(String path) throws IOException {
+ String fullpath = createFullPath(path);
+ try {
+ LOG.debug("ls {}", fullpath);
+ GetChildrenBuilder builder = curator.getChildren();
+ List 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 {
+ String fullpath = createFullPath(path);
+ try {
+ 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 toString()
+ * operation
+ * @return a class to dump the registry
+ */
+ @VisibleForTesting
+ public ZKPathDumper dumpPath() {
+ return new ZKPathDumper(curator, registryRoot);
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/services/RegistryBindingSource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/services/RegistryBindingSource.java
new file mode 100644
index 0000000..b97403d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/services/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.yarn.registry.client.services;
+
+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();
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/services/RegistryOperationsService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/services/RegistryOperationsService.java
new file mode 100644
index 0000000..c9701f8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/services/RegistryOperationsService.java
@@ -0,0 +1,203 @@
+/*
+ * 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.yarn.registry.client.services;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+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.security.AccessControlException;
+import org.apache.hadoop.yarn.registry.client.api.RegistryOperations;
+import org.apache.hadoop.yarn.registry.client.binding.RecordOperations;
+import static org.apache.hadoop.yarn.registry.client.binding.RegistryPathUtils.*;
+
+import org.apache.hadoop.yarn.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.yarn.registry.client.exceptions.InvalidPathnameException;
+import org.apache.hadoop.yarn.registry.client.exceptions.NoChildrenForEphemeralsException;
+import org.apache.hadoop.yarn.registry.client.types.CreateFlags;
+import org.apache.hadoop.yarn.registry.client.types.PersistencePolicies;
+import org.apache.hadoop.yarn.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.yarn.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.ArrayList;
+import java.util.List;
+
+/**
+ * The YARN ZK registry operations service.
+ *
+ * It's a YARN service: ephemeral nodes last as long as the client exists
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RegistryOperationsService extends CuratorService
+ implements RegistryOperations{
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(RegistryOperationsService.class);
+
+ private final RecordOperations.ServiceRecordMarshal serviceRecordMarshal
+ = new RecordOperations.ServiceRecordMarshal();
+
+ public static final String PERMISSIONS_REGISTRY_ROOT = "world:anyone:rwcda";
+ public static final String PERMISSIONS_REGISTRY_SYSTEM = "world:anyone:rwcda";
+ public static final String PERMISSIONS_REGISTRY_USERS = "world:anyone:rwcda";
+ public static final String PERMISSIONS_REGISTRY_USER = "world:anyone:rwcda";
+ private List userAcl;
+
+ public RegistryOperationsService(String name) {
+ this(name, null);
+ }
+
+ public RegistryOperationsService() {
+ this("RegistryOperationsService");
+ }
+
+ public RegistryOperationsService(String name,
+ RegistryBindingSource bindingSource) {
+ super(name, bindingSource);
+ }
+
+ @Override
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
+ userAcl = parseACLs(PERMISSIONS_REGISTRY_USERS);
+ }
+
+ public List getUserAcl() {
+ return userAcl;
+ }
+
+ protected void validatePath(String path) throws InvalidPathnameException {
+ RegistryPathUtils.validateElementsAsDNS(path);
+ }
+
+ @Override
+ public boolean mkdir(String path, boolean createParents) throws
+ PathNotFoundException,
+ NoChildrenForEphemeralsException,
+ AccessControlException,
+ InvalidPathnameException,
+ IOException {
+ validatePath(path);
+ return zkMkPath(path, CreateMode.PERSISTENT, createParents, getUserAcl());
+ }
+
+ @Override
+ public void create(String path,
+ ServiceRecord record,
+ int createFlags) throws
+ PathNotFoundException,
+ NoChildrenForEphemeralsException,
+ FileAlreadyExistsException,
+ AccessControlException,
+ InvalidPathnameException,
+ IOException {
+ Preconditions.checkArgument(record != null, "null record");
+ Preconditions.checkArgument(!Strings.isNullOrEmpty(record.id),
+ "empty record ID");
+ validatePath(path);
+ LOG.info("Registered at {} : {}", path, record);
+
+ boolean ephemeral = (createFlags & CreateFlags.EPHEMERAL) != 0;
+ CreateMode mode;
+ if (ephemeral) {
+ mode = CreateMode.EPHEMERAL;
+ Preconditions.checkArgument(
+ record.persistence == PersistencePolicies.EPHEMERAL,
+ "Ephemeral records can only be created if the record's persistence" +
+ " policy field is set to Ephemeral");
+ } else {
+ Preconditions.checkArgument(
+ record.persistence != PersistencePolicies.EPHEMERAL,
+ "The record's persistence field can only be set to Ephemeral if the" +
+ " create() operation requests an ephemeral entry");
+ mode = CreateMode.PERSISTENT;
+ }
+
+ byte[] bytes = serviceRecordMarshal.toByteswithHeader(record);
+ zkSet(path, mode, bytes, getUserAcl(),
+ ((createFlags & CreateFlags.OVERWRITE) != 0));
+ }
+
+ @Override
+ public ServiceRecord resolve(String path) throws
+ PathNotFoundException,
+ AccessControlException,
+ InvalidPathnameException,
+ IOException {
+ byte[] bytes = zkRead(path);
+ return serviceRecordMarshal.fromBytesWithHeader(path, bytes);
+ }
+
+ @Override
+ public RegistryPathStatus stat(String path) throws
+ PathNotFoundException,
+ AccessControlException,
+ InvalidPathnameException,
+ IOException {
+ validatePath(path);
+ Stat stat = zkStat(path);
+ RegistryPathStatus status = new RegistryPathStatus(
+ path,
+ stat.getCtime(),
+ stat.getDataLength(),
+ stat.getNumChildren());
+ LOG.debug("Stat {} => {}", path, status);
+ return status;
+ }
+
+ @Override
+ public RegistryPathStatus[] listDir(String path) throws
+ PathNotFoundException,
+ AccessControlException,
+ InvalidPathnameException,
+ IOException {
+ validatePath(path);
+ List childNames = zkList(path);
+ RegistryPathStatus[] results = new RegistryPathStatus[0];
+ int size = childNames.size();
+ ArrayList childList = new ArrayList(
+ size);
+ for (String childName : childNames) {
+ childList.add(stat(join(path, childName)));
+ }
+ return childList.toArray(new RegistryPathStatus[size]);
+ }
+
+ @Override
+ public void delete(String path, boolean recursive) throws
+ PathNotFoundException,
+ PathIsNotEmptyDirectoryException,
+ AccessControlException,
+ InvalidPathnameException,
+ IOException {
+ validatePath(path);
+ zkDelete(path, recursive, null);
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/AddressTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/AddressTypes.java
new file mode 100644
index 0000000..6251936
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/AddressTypes.java
@@ -0,0 +1,52 @@
+/*
+ * 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.yarn.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 {
+
+ /**
+ * Any other address
+ */
+ public static final String ADDRESS_OTHER = "";
+
+ /**
+ * URI entries
+ */
+ public static final String ADDRESS_URI = "uri";
+
+ /**
+ * hostname/port. A "/" is used so that IPv6 addresses can be
+ * easily split from the port declaration.
+ */
+ public static final String ADDRESS_HOSTNAME_AND_PORT = "host/port";
+
+ /**
+ * path /a/b/c style
+ */
+ public static final String ADDRESS_PATH = "path";
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/CreateFlags.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/CreateFlags.java
new file mode 100644
index 0000000..8d8702c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/CreateFlags.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.yarn.registry.client.types;
+
+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 CreateFlags {
+
+ /**
+ * Create the entry.. This is just "0" and can be "or"ed with anything
+ */
+ int CREATE = 0;
+
+ /**
+ * The entry is ephemeral, when this session is closed the entry
+ * will be deleted.
+ */
+ int EPHEMERAL = 1;
+
+ /**
+ * The entry should be created even if an existing entry is there.
+ */
+ int OVERWRITE = 2;
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/Endpoint.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/Endpoint.java
new file mode 100644
index 0000000..c7608b4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/Endpoint.java
@@ -0,0 +1,136 @@
+/*
+ * 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.yarn.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.yarn.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.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Description of a single service/component endpoint.
+ * It is designed to be marshalled as JSON
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class Endpoint {
+
+ /**
+ * 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> addresses;
+
+ public Endpoint() {
+ }
+
+
+ /**
+ * 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... addrs) {
+ this.api = api;
+ this.addressType = addressType;
+ this.protocolType = protocolType;
+ this.addresses = new ArrayList>();
+ if (addrs != null) {
+ Collections.addAll(addresses, 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
+ */
+ public Endpoint(String api,
+ String protocolType,
+ URI... uris) {
+ this.api = api;
+ this.addressType = AddressTypes.ADDRESS_URI;
+
+ this.protocolType = protocolType;
+ List> addrs = new ArrayList>(uris.length);
+ for (URI uri : uris) {
+ ArrayList elt = new ArrayList(1);
+ 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('\'');
+ if (addresses != null) {
+ sb.append(", address count=").append(addresses.size());
+ } else {
+ sb.append(", null address list=");
+ }
+ sb.append('}');
+ return sb.toString();
+ }
+
+ /**
+ * Validate the record by checking for null fields
+ */
+ 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");
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/PersistencePolicies.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/PersistencePolicies.java
new file mode 100644
index 0000000..e8162ed
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/PersistencePolicies.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.yarn.registry.client.types;
+
+/**
+ * Persistence policies for {@link ServiceRecord#persistence}
+ */
+
+public interface PersistencePolicies {
+
+ /**
+ * The record persists until removed manually: {@value}.
+ */
+ int PERMANENT = 0;
+
+ /**
+ * Remove when the YARN cluster is restarted: {@value}.
+ * This does not mean on HA failover; it means after a cluster stop/start.
+ */
+ int CLUSTER_RESTART = 1;
+
+ /**
+ * Remove when the YARN application defined in the id field
+ * terminates: {@value}.
+ */
+ int APPLICATION = 2;
+
+ /**
+ * Remove when the current YARN application attempt ID finishes: {@value}.
+ */
+ int APPLICATION_ATTEMPT = 3;
+
+ /**
+ * Remove when the YARN container in the ID field finishes.
+ */
+ int CONTAINER = 4;
+
+ /**
+ * Automatic deletion when the session is closed/times out: {@value}.
+ * This is implemented at the ZK layer, not in the RM.
+ */
+ int EPHEMERAL = 5;
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/ProtocolTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/ProtocolTypes.java
new file mode 100644
index 0000000..09fec3b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/ProtocolTypes.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.yarn.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 {
+
+ String PROTOCOL_HADOOP_IPC = "hadoop/IPC";
+ String PROTOCOL_HADOOP_IPC_PROTOBUF = "hadoop/protobuf";
+ String PROTOCOL_IIOP = "IIOP";
+ String PROTOCOL_REST = "REST";
+ String PROTOCOL_RMI = "RMI";
+ String PROTOCOL_SUN_RPC = "sunrpc";
+ String PROTOCOL_THRIFT = "thrift";
+ String PROTOCOL_TCP = "tcp";
+ String PROTOCOL_UDP = "udp";
+ String PROTOCOL_UNKNOWN = "";
+ String PROTOCOL_WEBUI = "webui";
+ String PROTOCOL_WSAPI = "WS";
+ String PROTOCOL_ZOOKEEPER_BINDING = "zookeeper";
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/RegistryPathStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/RegistryPathStatus.java
new file mode 100644
index 0000000..04e5136
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/RegistryPathStatus.java
@@ -0,0 +1,90 @@
+/*
+ * 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.yarn.registry.client.types;
+
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Output of a stat() call
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class RegistryPathStatus {
+
+ /**
+ * Path in the registry to this entry
+ */
+ public final String path;
+ public final long time;
+ public final long size;
+ public final int children;
+
+
+ public RegistryPathStatus(String path,
+ long time,
+ long size, int children) {
+ this.path = path;
+ this.time = time;
+ this.size = size;
+ this.children = children;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ RegistryPathStatus status = (RegistryPathStatus) o;
+
+ 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;
+ }
+
+ @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();
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/ServiceRecord.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/ServiceRecord.java
new file mode 100644
index 0000000..8f264ed
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/client/types/ServiceRecord.java
@@ -0,0 +1,148 @@
+/*
+ * 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.yarn.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.JsonIgnoreProperties;
+
+import java.util.ArrayList;
+import java.util.List;
+
+
+/**
+ * JSON-marshallable description of a single component
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class ServiceRecord {
+
+ /**
+ * The time the service was registered -as seen by the service making
+ * the registration request.
+ */
+ public long registrationTime;
+
+ /**
+ * ID. For containers: container ID. For application instances, application ID.
+ */
+ public String id;
+
+ /**
+ * Description string
+ */
+ public String description;
+
+ /**
+ * The persistence attribute defines when a record and any child
+ * entries may be deleted.
+ * {@link PersistencePolicies}
+ */
+ public int persistence = PersistencePolicies.PERMANENT;
+
+ /**
+ * List of endpoints intended to of use to external callers
+ */
+ public List external = new ArrayList();
+
+ /**
+ * List of internal endpoints
+ */
+ public List internal = new ArrayList();
+
+
+ /**
+ * Create a service record with no ID, description or registration time.
+ * Endpoint lists are set to empty lists.
+ */
+ public ServiceRecord() {
+ }
+
+ /**
+ * Create a service record ... sets the registration time to the current
+ * system time.
+ * @param id service ID
+ * @param description description
+ * @param persistence persistence policy
+ */
+ public ServiceRecord(String id, String description, int persistence) {
+ this.id = id;
+ this.description = description;
+ this.persistence = persistence;
+ this.registrationTime = System.currentTimeMillis();
+ }
+
+ public void addExternalEndpoint(Endpoint endpoint) {
+ Preconditions.checkArgument(endpoint != null);
+ endpoint.validate();
+ external.add(endpoint);
+ }
+
+ public void addInternalEndpoint(Endpoint endpoint) {
+ Preconditions.checkArgument(endpoint != null);
+ endpoint.validate();
+ internal.add(endpoint);
+ }
+
+ public Endpoint getInternalEndpoint(String api) {
+ return findByAPI(internal, api);
+ }
+
+ public Endpoint getExternalEndpoint(String api) {
+ return findByAPI(external, api);
+ }
+
+ /**
+ * Find an endpoint by its API
+ * @param list list
+ * @param api api name
+ * @return
+ */
+ private Endpoint findByAPI(List 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("id='").append(id).append('\'');
+ sb.append(", persistence='").append(persistence).append('\'');
+ 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).append("; ");
+ }
+
+ sb.append('}');
+ sb.append('}');
+ return sb.toString();
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/server/services/MicroZookeeperService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/server/services/MicroZookeeperService.java
new file mode 100644
index 0000000..aeda7ab
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/server/services/MicroZookeeperService.java
@@ -0,0 +1,209 @@
+/*
+ * 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.yarn.registry.server.services;
+
+import com.google.common.base.Preconditions;
+import org.apache.curator.ensemble.fixed.FixedEnsembleProvider;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.registry.client.services.BindingInformation;
+import org.apache.hadoop.yarn.registry.client.services.RegistryBindingSource;
+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.FileNotFoundException;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.net.InetAddress;
+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,
+ * once started. Until start() 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 CompositeService, 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.
+ */
+public class MicroZookeeperService
+ extends AbstractService
+ implements RegistryBindingSource, RegistryConstants {
+
+
+ private static final Logger
+ LOG = LoggerFactory.getLogger(MicroZookeeperService.class);
+
+ private File dataDir;
+ private int tickTime;
+ private int port;
+
+ private ServerCnxnFactory factory;
+ private BindingInformation binding;
+
+ /**
+ * 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 machine doesn't know its own address
+ */
+ private InetSocketAddress getAddress(int port) throws UnknownHostException {
+ return new InetSocketAddress(InetAddress.getLocalHost(),
+ 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 = getConfig().getInt(KEY_ZKSERVICE_PORT, 0);
+ tickTime = getConfig().getInt(KEY_ZKSERVICE_TICK_TIME,
+ ZooKeeperServer.DEFAULT_TICK_TIME);
+ String datapathname = getConfig().getTrimmed(KEY_ZKSERVICE_DATADIR, "");
+ if (datapathname.isEmpty()) {
+ dataDir = File.createTempFile("zkservice", ".dir");
+ dataDir.delete();
+ } else {
+ dataDir = new File(datapathname);
+ FileUtil.fullyDelete(dataDir);
+ }
+ LOG.debug("Data directory is {}", dataDir);
+ // the exit code here is ambigious
+ dataDir.mkdirs();
+ // so: verify the path is there and a directory
+ if (!dataDir.exists()) {
+ throw new FileNotFoundException("failed to create directory " + dataDir);
+ }
+ if (!dataDir.isDirectory()) {
+
+ throw new IOException(
+ "Path " + dataDir + " exists but is not a directory "
+ + " isDir()=" + dataDir.isDirectory()
+ + " isFile()=" + dataDir.isFile());
+ }
+ super.serviceInit(conf);
+ }
+
+ /**
+ * Startup: start ZK. It is only after this that
+ * the binding information is valid.
+ * @throws Exception
+ */
+ @Override
+ protected void serviceStart() throws Exception {
+ 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: {}", 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 + "\"";
+
+ // 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;
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/server/services/RMRegistryOperationsService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/server/services/RMRegistryOperationsService.java
new file mode 100644
index 0000000..5b9ee43
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/yarn/registry/server/services/RMRegistryOperationsService.java
@@ -0,0 +1,398 @@
+/*
+ * 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.yarn.registry.server.services;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
+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.yarn.registry.client.binding.BindingUtils;
+import org.apache.hadoop.yarn.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.yarn.registry.client.services.RegistryBindingSource;
+import org.apache.hadoop.yarn.registry.client.services.RegistryOperationsService;
+import org.apache.hadoop.yarn.registry.client.types.PersistencePolicies;
+import org.apache.hadoop.yarn.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Extends the registry operations with extra support for resource management
+ * operations, including creating and cleaning up the registry.
+ *
+ * These actions are all implemented as event handlers to operations
+ * which come from the RM.
+ *
+ * This service is expected to be executed by a user with the permissions
+ * to manipulate the entire registry,
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RMRegistryOperationsService extends RegistryOperationsService {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(RMRegistryOperationsService.class);
+
+ private List rootRegistryACL;
+
+ private List userAcl;
+
+ private PurgePolicy purgeOnCompletionPolicy = PurgePolicy.PurgeAll;
+
+ public RMRegistryOperationsService(String name) {
+ this(name, null);
+ }
+
+ public RMRegistryOperationsService(String name,
+ RegistryBindingSource bindingSource) {
+ super(name, bindingSource);
+ }
+
+ @Override
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
+ userAcl = parseACLs(PERMISSIONS_REGISTRY_USERS);
+ }
+
+ /**
+ * Start the service, including creating base directories with permissions
+ * @throws Exception
+ */
+ @Override
+ protected void serviceStart() throws Exception {
+ super.serviceStart();
+
+ // create the root directories
+ createRegistryPaths();
+ }
+
+ /**
+ * Create the initial registry paths
+ * @throws IOException any failure
+ */
+ @VisibleForTesting
+ public void createRegistryPaths() throws IOException {
+ // create the root directories
+ rootRegistryACL = getACLs(KEY_REGISTRY_ZK_ACL, PERMISSIONS_REGISTRY_ROOT);
+ maybeCreate("", CreateMode.PERSISTENT, rootRegistryACL, false);
+
+ maybeCreate(PATH_USERS, CreateMode.PERSISTENT,
+ parseACLs(PERMISSIONS_REGISTRY_USERS), false);
+ maybeCreate(PATH_SYSTEM_SERVICES, CreateMode.PERSISTENT,
+ parseACLs(PERMISSIONS_REGISTRY_SYSTEM), false);
+ }
+
+ /**
+ * Create the path for a user
+ * @param username username
+ * @throws IOException any failure
+ */
+ public void createHomeDirectory(String username) throws IOException {
+ String path = homeDir(username);
+ maybeCreate(path, CreateMode.PERSISTENT,
+ createAclForUser(username), 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 BindingUtils.userPath(username);
+ }
+
+ /**
+ * Set up the ACL for the user.
+ * @param username user name
+ * @return an ACL list
+ * @throws IOException ACL creation/parsing problems
+ */
+ private List createAclForUser(String username) throws IOException {
+ return userAcl;
+ }
+
+ 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(ApplicationAttemptId 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 an application attempt is completed
+ * @param attemptId application attempt ID
+ * @throws IOException problems
+ */
+ public void onApplicationAttemptCompleted(ApplicationAttemptId attemptId)
+ throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Application Attempt {} completed, purging application-level records",
+ attemptId);
+ }
+ purgeRecordsQuietly("/",
+ attemptId.toString(),
+ PersistencePolicies.APPLICATION_ATTEMPT);
+ }
+
+ /**
+ * Actions to take when an application attempt is completed
+ * @param attemptId application ID
+ * @throws IOException problems
+ */
+ public void onApplicationUnregistered(ApplicationAttemptId attemptId)
+ throws IOException {
+ LOG.info("Application attempt {} unregistered, purging app attempt records",
+ attemptId);
+ purgeRecordsQuietly("/",
+ attemptId.toString(),
+ PersistencePolicies.APPLICATION_ATTEMPT);
+ }
+/**
+ * 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);
+ purgeRecordsQuietly("/",
+ 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 {
+ createHomeDirectory(user);
+ }
+
+ /**
+ * 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
+ purgeRecordsQuietly("/",
+ containerId.getApplicationAttemptId().toString(),
+ PersistencePolicies.APPLICATION_ATTEMPT);
+
+ // also treat as a container finish to remove container
+ // level records for the AM container
+ onContainerFinished(containerId);
+ }
+
+
+ /**
+ * 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);
+ purgeRecordsQuietly("/",
+ id.toString(),
+ PersistencePolicies.CONTAINER);
+ }
+ /**
+ * Policy to purge entries
+ */
+ public enum PurgePolicy {
+ PurgeAll,
+ FailOnChildren,
+ SkipOnChildren
+ }
+
+ /**
+ * Purge all matching records under a base path -logging problems at INFO.
+ *
+ *
Uses a depth first search
+ *
A match is on ID and persistence policy, or, if policy==-1, any match
+ *
If a record matches then it is deleted without any child searches
+ *
Deletions will be asynchronous if a callback is provided
+ *
+ * @param path base path
+ * @param id ID for service record.id
+ * @param persistencePolicyMatch ID for the persistence policy to match: no match, no delete.
+ * If set to to -1 or below, " don't check"
+ */
+ private void purgeRecordsQuietly(String path,
+ String id,
+ int persistencePolicyMatch) {
+ try {
+ LOG.info("Purging records under {} with ID {} and policy {}: {}",
+ path, id, persistencePolicyMatch);
+ purgeRecords(path, id, persistencePolicyMatch, purgeOnCompletionPolicy,
+ new DeleteCompletionCallback());
+ } catch (IOException e) {
+ LOG.info("Error while purging records under {} with ID {} and policy {}: {}",
+ path, id, persistencePolicyMatch, e, e);
+ }
+ }
+
+ /**
+ * Recursive operation to purge all matching records under a base path.
+ *
+ *
Uses a depth first search
+ *
A match is on ID and persistence policy, or, if policy==-1, any match
+ *
If a record matches then it is deleted without any child searches
+ *
Deletions will be asynchronous if a callback is provided
+ *
+ *
+ * @param path base path
+ * @param id ID for service record.id
+ * @param persistencePolicyMatch ID for the persistence policy to match: no match, no delete.
+ * If set to to -1 or below, " don't check"
+ * @param purgePolicy what to do if there is a matching record with children
+ * @return the number of calls to the zkDelete() operation. This is purely for
+ * testing.
+ * @throws IOException problems
+ * @throws PathIsNotEmptyDirectoryException if an entry cannot be deleted
+ * as his children and the purge policy is FailOnChildren
+ */
+ @VisibleForTesting
+ public int purgeRecords(String path,
+ String id,
+ int persistencePolicyMatch,
+ PurgePolicy purgePolicy,
+ BackgroundCallback callback) throws IOException {
+ Preconditions.checkArgument(StringUtils.isNotEmpty(path),
+ "Empty 'path' argument");
+ Preconditions.checkArgument(StringUtils.isNotEmpty(id),
+ "Empty 'id' argument");
+
+ // list this path's children
+ RegistryPathStatus[] entries = listDir(path);
+
+ boolean toDelete = false;
+ // look at self to see if it has a service record
+ try {
+ ServiceRecord serviceRecord = resolve(path);
+ // there is now an entry here.
+ toDelete = serviceRecord.id.equals(id)
+ && (persistencePolicyMatch < 0
+ || serviceRecord.persistence == persistencePolicyMatch);
+ } catch (EOFException ignored) {
+ // ignore
+ } catch (InvalidRecordException ignored) {
+ // ignore
+ }
+
+ if (toDelete && entries.length > 0) {
+ 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
+ toDelete = false;
+ break;
+ case PurgeAll:
+ // mark for deletion
+ toDelete = true;
+ entries = new RegistryPathStatus[0];
+ break;
+ case FailOnChildren:
+ throw new PathIsNotEmptyDirectoryException(path);
+ }
+ }
+
+ int deleteOps = 0;
+
+ if(toDelete) {
+ deleteOps++;
+ zkDelete(path, true, callback);
+ }
+
+ // now go through the children
+ for (RegistryPathStatus status : entries) {
+ deleteOps += purgeRecords(status.path,
+ id,
+ persistencePolicyMatch,
+ purgePolicy,
+ callback);
+ }
+
+ return deleteOps;
+ }
+
+
+ /**
+ * Callback for delete operations completing
+ */
+ protected static class DeleteCompletionCallback implements BackgroundCallback {
+ @Override
+ public void processResult(CuratorFramework client,
+ CuratorEvent event) throws
+ Exception {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Delete event "+ event.toString());
+ }
+ }
+ }
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/resources/webapps/registry/.keep b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/resources/webapps/registry/.keep
new file mode 100644
index 0000000..e69de29
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/resources/webapps/registry/index.html b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/resources/webapps/registry/index.html
new file mode 100644
index 0000000..3a9d246
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/resources/webapps/registry/index.html
@@ -0,0 +1,28 @@
+
+
+
+
+
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/site/markdown/architecture.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/site/markdown/architecture.md
new file mode 100644
index 0000000..3846b6f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/site/markdown/architecture.md
@@ -0,0 +1,38 @@
+
+
+# Architecture
+
+ZK is used as the underlying system of record. Writes update entries in the
+ZK namespace, reads query it
+
+1. all writes are asynchronous and eventually consistent across the ZK quorum.
+All that is guaranteed is that updates from a single client are processed in
+the order in which they are submitted.
+
+1. The API appears RESTful even when the implementation is not. This is to
+offer a consistent view for the (currently Read-only) REST view as well
+as the read/write ZK operations.
+
+1. In the hierarchy of entries, *components* may be ephemeral, but all other
+nodes (znodes) are not. This is actually a requirement of ZooKeeper, but is
+required here to support the scenario "service temporarily offline, registration data
+still visible". There is, after all, no prerequisite that a service is actually
+running for a client to communicate with it -only that the service must
+be running for it to process the data and/or issue a response.
+
+## Hierarchy
+
+ $base/users/$user/$service-class/$instance/components/*
+ $base/users/$user/$service-class/$instance -> instance data
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/AbstractZKRegistryTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/AbstractZKRegistryTest.java
new file mode 100644
index 0000000..b10c967
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/AbstractZKRegistryTest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.yarn.registry;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.registry.server.services.MicroZookeeperService;
+import org.apache.zookeeper.common.PathUtils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+
+import java.io.File;
+import java.io.IOException;
+
+public class AbstractZKRegistryTest extends Assert {
+
+ protected static MicroZookeeperService zookeeper;
+
+ @Rule
+ public final Timeout testTimeout = new Timeout(10000);
+
+ @Rule
+ public TestName methodName = new TestName();
+
+ @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(RegistryConstants.KEY_ZKSERVICE_DATADIR, zkDir.getAbsolutePath());
+ zookeeper.init(conf);
+ zookeeper.start();
+ }
+
+ @AfterClass
+ public static void destroyZKServer() throws IOException {
+ zookeeper.close();
+ }
+
+ public static void assertValidZKPath(String path) {
+ try {
+ PathUtils.validatePath(path);
+ } catch (IllegalArgumentException e) {
+ throw new IllegalArgumentException("Invalid Path " + path + ": " + e, e);
+ }
+ }
+
+ /**
+ * 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();
+ }
+
+ protected 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;
+ }
+}
+
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/binding/TestMarshalling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/binding/TestMarshalling.java
new file mode 100644
index 0000000..43c5e28
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/binding/TestMarshalling.java
@@ -0,0 +1,88 @@
+/*
+ * 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.yarn.registry.client.binding;
+
+import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+
+import java.io.EOFException;
+
+/**
+ * Test record marshalling
+ */
+public class TestMarshalling extends Assert {
+ @Rule
+ public final Timeout testTimeout = new Timeout(10000);
+ @Rule
+ public TestName methodName = new TestName();
+ private static RecordOperations.ServiceRecordMarshal marshal;
+
+ @BeforeClass
+ public static void setupClass() {
+ marshal = new RecordOperations.ServiceRecordMarshal();
+ }
+
+ @Test
+ public void testRoundTrip() throws Throwable {
+ ServiceRecord record = new ServiceRecord("01", "description", 0);
+ byte[] bytes = marshal.toBytes(record);
+ ServiceRecord r2 = marshal.fromBytes("", bytes, 0);
+ assertEquals(record.id, r2.id);
+ assertEquals(record.persistence, r2.persistence);
+ assertEquals(record.description, r2.description);
+ }
+
+ @Test
+ public void testRoundTripHeaders() throws Throwable {
+ ServiceRecord record = new ServiceRecord("01", "description", 1);
+ byte[] bytes = marshal.toByteswithHeader(record);
+ ServiceRecord r2 = marshal.fromBytesWithHeader("", bytes);
+ assertEquals(record.id, r2.id);
+ assertEquals(record.persistence, r2.persistence);
+ assertEquals(record.description, r2.description);
+ }
+
+ @Test(expected = InvalidRecordException.class)
+ public void testRoundTripBadHeaders() throws Throwable {
+ ServiceRecord record = new ServiceRecord("01", "description", 0);
+ byte[] bytes = marshal.toByteswithHeader(record);
+ bytes[1] = 0x01;
+ marshal.fromBytesWithHeader("src", bytes);
+ }
+
+ @Test(expected = InvalidRecordException.class)
+ public void testUnmarshallHeaderTooShort() throws Throwable {
+ marshal.fromBytesWithHeader("src", new byte[]{'a'});
+ }
+
+ @Test(expected = EOFException.class)
+ public void testUnmarshallNoBody() throws Throwable {
+ byte[] bytes = RegistryConstants.RECORD_HEADER;
+ marshal.fromBytesWithHeader("src", bytes);
+ }
+
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/binding/TestRegistryPathUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/binding/TestRegistryPathUtils.java
new file mode 100644
index 0000000..9eb6585
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/binding/TestRegistryPathUtils.java
@@ -0,0 +1,165 @@
+/*
+ * 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.yarn.registry.client.binding;
+
+import static org.apache.hadoop.yarn.registry.client.binding.RegistryPathUtils.*;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.yarn.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", "/");
+ }
+
+ @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 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(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
+ }
+ }
+
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/services/CuratorEventCatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/services/CuratorEventCatcher.java
new file mode 100644
index 0000000..a4ca1c3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/services/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.yarn.registry.client.services;
+
+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.
+ */
+class CuratorEventCatcher implements BackgroundCallback {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(CuratorEventCatcher.class);
+
+ public final BlockingQueue
+ events = new LinkedBlockingQueue(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();
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/services/TestCuratorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/services/TestCuratorService.java
new file mode 100644
index 0000000..5e2a320
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/services/TestCuratorService.java
@@ -0,0 +1,250 @@
+/*
+ * 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.yarn.registry.client.services;
+
+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.yarn.registry.AbstractZKRegistryTest;
+import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.registry.server.services.RMRegistryOperationsService;
+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 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 = curatorService.getACLs(RegistryConstants.KEY_REGISTRY_ZK_ACL,
+ "world:anyone:rwcda");
+ List rootACL = curatorService.getACLs(
+ RegistryConstants.KEY_REGISTRY_ZK_ACL,
+ RMRegistryOperationsService.PERMISSIONS_REGISTRY_ROOT);
+ curatorService.maybeCreate("", CreateMode.PERSISTENT, rootACL, true);
+ }
+
+ @Test
+ public void testLs() throws Throwable {
+ curatorService.zkList("/");
+ }
+
+ @Test(expected = PathNotFoundException.class)
+ public void testLsNotFound() throws Throwable {
+ List 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);
+ }
+
+ 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));
+ assertFalse(curatorService.maybeCreate("/p3", CreateMode.PERSISTENT));
+ }
+
+ @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);
+ }
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/services/TestMicroZookeeperService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/services/TestMicroZookeeperService.java
new file mode 100644
index 0000000..aba3107
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/services/TestMicroZookeeperService.java
@@ -0,0 +1,61 @@
+/*
+ * 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.yarn.registry.client.services;
+
+import org.apache.hadoop.service.ServiceOperations;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.registry.server.services.MicroZookeeperService;
+import org.junit.After;
+import org.junit.AfterClass;
+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();
+ }
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/services/TestRegistryOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/services/TestRegistryOperations.java
new file mode 100644
index 0000000..c72b8de
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/yarn/registry/client/services/TestRegistryOperations.java
@@ -0,0 +1,671 @@
+/*
+ * 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.yarn.registry.client.services;
+
+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.yarn.registry.AbstractZKRegistryTest;
+import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.registry.client.api.RegistryOperations;
+import static org.apache.hadoop.yarn.registry.client.binding.RegistryTypeUtils.*;
+
+import org.apache.hadoop.yarn.registry.client.binding.RecordOperations;
+import org.apache.hadoop.yarn.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.yarn.registry.client.binding.RegistryTypeUtils;
+import org.apache.hadoop.yarn.registry.client.binding.ZKPathDumper;
+import org.apache.hadoop.yarn.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.yarn.registry.client.exceptions.NoChildrenForEphemeralsException;
+import org.apache.hadoop.yarn.registry.client.types.AddressTypes;
+import org.apache.hadoop.yarn.registry.client.types.CreateFlags;
+import org.apache.hadoop.yarn.registry.client.types.Endpoint;
+import org.apache.hadoop.yarn.registry.client.types.PersistencePolicies;
+import org.apache.hadoop.yarn.registry.client.types.ProtocolTypes;
+import org.apache.hadoop.yarn.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.yarn.registry.client.types.ServiceRecord;
+import org.apache.hadoop.yarn.registry.server.services.RMRegistryOperationsService;
+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.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Map;
+
+public class TestRegistryOperations extends AbstractZKRegistryTest {
+ 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(TestRegistryOperations.class);
+
+ private final RecordOperations.ServiceRecordMarshal recordMarshal =
+ new RecordOperations.ServiceRecordMarshal();
+
+ private RMRegistryOperationsService registry;
+
+ private RegistryOperations operations;
+
+
+ @Before
+ public void setupClient() throws IOException {
+ registry = new RMRegistryOperationsService("yarnRegistry");
+ registry.init(createRegistryConfiguration());
+ registry.start();
+ registry.createRegistryPaths();
+ operations = registry;
+ operations.delete(ENTRY_PATH, true);
+ }
+
+ @After
+ public void teardownClient() {
+ ServiceOperations.stop(registry);
+ }
+
+ /**
+ * Add some endpoints
+ * @param entry entry
+ */
+ protected void addSampleEndpoints(ServiceRecord entry, String hostname) throws
+ URISyntaxException {
+ 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)));
+
+ }
+
+ /**
+ * General code to validate bits of a component/service entry built iwth
+ * {@link #addSampleEndpoints(ServiceRecord, String)}
+ * @param record instance to check
+ */
+ protected void validateEntry(ServiceRecord record) {
+ assertNotNull("null service record", record);
+ List 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> addressList = webhdfs.addresses;
+ List 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());
+
+ }
+
+
+ /**
+ * 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,
+ int persistence)
+ throws IOException, URISyntaxException {
+ ServiceRecord record = buildExampleServiceEntry(persistence);
+
+ registry.mkdir(RegistryPathUtils.parentOf(path), true);
+ operations.create(path, record, createFlags);
+ return record;
+ }
+
+ /**
+ * Create a service entry with the sample endpoints
+ * @param persistence persistence policy
+ * @return the record
+ * @throws IOException on a failure
+ */
+ private ServiceRecord buildExampleServiceEntry(int persistence) throws
+ IOException,
+ URISyntaxException {
+ List acls = registry.parseACLs("world:anyone:rwcda");
+ ServiceRecord record = new ServiceRecord();
+ record.id = "example-0001";
+ record.persistence = persistence;
+ record.description = methodName.getMethodName();
+ record.registrationTime = System.currentTimeMillis();
+ addSampleEndpoints(record, "namenode");
+ return record;
+ }
+
+
+ public void assertMatches(Endpoint endpoint,
+ String addressType,
+ String protocolType,
+ String api) {
+ assertNotNull(endpoint);
+ assertEquals(addressType, endpoint.addressType);
+ assertEquals(protocolType, endpoint.protocolType);
+ assertEquals(api, endpoint.api);
+ }
+
+ public void assertMatches(ServiceRecord written, ServiceRecord resolved) {
+ assertEquals(written.id, resolved.id);
+ assertEquals(written.registrationTime, resolved.registrationTime);
+ assertEquals(written.description, resolved.description);
+ assertEquals(written.persistence, resolved.persistence);
+ }
+
+
+ /**
+ * Assert a path exists
+ * @param path
+ * @throws IOException
+ */
+ public void assertPathExists(String path) throws IOException {
+ operations.stat(path);
+ }
+
+ public void assertPathNotFound(String path) throws IOException {
+ try {
+ operations.stat(path);
+ fail("Path unexpectedly found: " + path);
+ } catch (PathNotFoundException e) {
+
+ }
+ }
+
+ public void assertResolves(String path) throws IOException {
+ operations.resolve(path);
+ }
+
+
+ public Endpoint findEndpoint(ServiceRecord record,
+ String api, boolean external, int addressElements, int elementSize) {
+ 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",
+ elementSize, epr.addresses.get(0).size());
+ return epr;
+ }
+ List 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);
+ return null;
+ }
+
+
+ public void log(String name, ServiceRecord record) throws
+ IOException {
+ LOG.info(" {} = \n{}\n", name, recordMarshal.toJson(record));
+ }
+
+ @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 {
+ ServiceRecord written = putExampleServiceEntry(ENTRY_PATH, 0);
+ RegistryPathStatus stat = operations.stat(ENTRY_PATH);
+ assertTrue(stat.size > 0);
+ assertTrue(stat.time > 0);
+ assertEquals(ENTRY_PATH, stat.path);
+ }
+
+ @Test
+ public void testLsParent() throws Throwable {
+ ServiceRecord written = putExampleServiceEntry(ENTRY_PATH, 0);
+ RegistryPathStatus stat = operations.stat(ENTRY_PATH);
+
+ RegistryPathStatus[] statuses =
+ operations.listDir(PARENT_PATH);
+ assertEquals(1, statuses.length);
+ assertEquals(stat, statuses[0]);
+
+ Map records =
+ RecordOperations.extractServiceRecords(operations, statuses);
+ assertEquals(1, records.size());
+ ServiceRecord record = records.get(ENTRY_PATH);
+ assertMatches(written, record);
+ }
+
+ @Test
+ public void testDeleteNonEmpty() throws Throwable {
+ ServiceRecord written = putExampleServiceEntry(ENTRY_PATH, 0);
+ RegistryPathStatus stat = operations.stat(ENTRY_PATH);
+ try {
+ operations.delete(PARENT_PATH, false);
+ fail("Expected a failure");
+ } catch (PathIsNotEmptyDirectoryException e) {
+
+ }
+ operations.delete(PARENT_PATH, true);
+
+ }
+
+ @Test(expected = PathNotFoundException.class)
+ public void testStatEmptyPath() throws Throwable {
+ RegistryPathStatus stat = operations.stat(ENTRY_PATH);
+ }
+
+ @Test(expected = PathNotFoundException.class)
+ public void testLsEmptyPath() throws Throwable {
+ RegistryPathStatus[] statuses =
+ operations.listDir(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.mkdir(path, false);
+ RegistryPathStatus stat = operations.stat(path);
+ fail("Got a status " + stat);
+ } catch (PathNotFoundException expected) {
+
+ }
+ }
+
+ @Test
+ public void testDoubleMkdir() throws Throwable {
+
+ operations.mkdir(USERPATH, false);
+ String path = USERPATH +"newentry";
+ assertTrue(operations.mkdir(path, false));
+ RegistryPathStatus stat = operations.stat(path);
+ assertFalse(operations.mkdir(path, false));
+ }
+
+ @Test
+ public void testPutNoParent() throws Throwable {
+ ServiceRecord record = new ServiceRecord();
+ record.id = "testPutNoParent";
+ String path = "/path/without/parent";
+ try {
+ operations.create(path, record, 0);
+ // didn't get a failure
+ // trouble
+ RegistryPathStatus stat = operations.stat(path);
+ fail("Got a status " + stat);
+ } catch (PathNotFoundException expected) {
+ }
+ }
+
+ @Test(expected = PathNotFoundException.class)
+ public void testPutNoParent2() throws Throwable {
+ ServiceRecord record = new ServiceRecord();
+ record.id = "testPutNoParent";
+ String path = "/path/without/parent";
+ operations.create(path, record, 0);
+ }
+
+ @Test
+ public void testStatPathThatHasNoEntry() throws Throwable {
+ String empty = "/empty";
+ operations.mkdir(empty, false);
+ RegistryPathStatus stat = operations.stat(empty);
+ }
+
+
+ @Test
+ public void testResolvePathThatHasNoEntry() throws Throwable {
+ String empty = "/empty2";
+ operations.mkdir(empty, false);
+ try {
+ ServiceRecord record = operations.resolve(empty);
+ fail("expected an exception");
+ } catch (InvalidRecordException expected) {
+
+ }
+ }
+
+
+ @Test
+ public void testOverwrite() throws Throwable {
+ ServiceRecord written = putExampleServiceEntry(ENTRY_PATH, 0);
+ ServiceRecord resolved1 = operations.resolve(ENTRY_PATH);
+ resolved1.description = "resolved1";
+ try {
+ operations.create(ENTRY_PATH, resolved1, 0);
+ fail("overwrite succeeded when it should have failed");
+ } catch (FileAlreadyExistsException expected) {
+
+ }
+
+ // verify there's no changed
+ ServiceRecord resolved2 = operations.resolve(ENTRY_PATH);
+ assertMatches(written, resolved2);
+ operations.create(ENTRY_PATH, resolved1, CreateFlags.OVERWRITE);
+ ServiceRecord resolved3 = operations.resolve(ENTRY_PATH);
+ assertMatches(resolved1, resolved3);
+ }
+
+ /**
+ * 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 = new ServiceRecord(appId,
+ "tomcat-based web application",
+ PersistencePolicies.APPLICATION);
+ webapp.addExternalEndpoint(restEndpoint("www",
+ new URI("http","//loadbalancer/", null)));
+
+ ServiceRecord comp1 = new ServiceRecord(cid1, null,
+ PersistencePolicies.EPHEMERAL);
+ 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 = new ServiceRecord(cid2, null,
+ PersistencePolicies.CONTAINER);
+ comp2.addExternalEndpoint(restEndpoint("www",
+ new URI("http", "//rack1server28:35881",null)));
+ comp2.addInternalEndpoint(
+ inetAddrEndpoint("jmx", "JMX", "rack1server28", 35882));
+
+ operations.mkdir(USERPATH, false);
+ operations.create(appPath, webapp, CreateFlags.OVERWRITE);
+ String components = appPath + RegistryConstants.SUBPATH_COMPONENTS + "/";
+ operations.mkdir(components, false);
+ String dns1 = RegistryPathUtils.encodeYarnID(cid1);
+ String dns1path = components + dns1;
+ operations.create(dns1path, comp1, CreateFlags.EPHEMERAL);
+ String dns2 = RegistryPathUtils.encodeYarnID(cid2);
+ String dns2path = components + dns2;
+ operations.create(dns2path, comp2, CreateFlags.CREATE );
+
+ ZKPathDumper pathDumper = registry.dumpPath();
+ LOG.info(pathDumper.toString());
+
+ log("tomcat", webapp);
+ log(dns1, comp1);
+ log(dns2, comp2);
+
+ ServiceRecord dns1resolved = operations.resolve(dns1path);
+ assertEquals("Persistence policies on resolved entry",
+ PersistencePolicies.EPHEMERAL, dns1resolved.persistence);
+
+
+ RegistryPathStatus[] componentStats = operations.listDir(components);
+ assertEquals(2, componentStats.length);
+ Map records =
+ RecordOperations.extractServiceRecords(operations, componentStats);
+ assertEquals(2, records.size());
+ ServiceRecord retrieved1 = records.get(dns1path);
+ log(retrieved1.id, retrieved1);
+ assertMatches(dns1resolved, retrieved1);
+ assertEquals(PersistencePolicies.EPHEMERAL, retrieved1.persistence);
+
+ // create a listing under components/
+ operations.mkdir(components + "subdir", false);
+ RegistryPathStatus[] componentStatsUpdated = operations.listDir(components);
+ assertEquals(3, componentStatsUpdated.length);
+ Map recordsUpdated =
+ RecordOperations.extractServiceRecords(operations, componentStats);
+ assertEquals(2, recordsUpdated.size());
+
+
+
+ // now do some deletions.
+
+ // synchronous delete container ID 2
+
+ // fail if the app policy is chosen
+ assertEquals(0, registry.purgeRecords("/", cid2,
+ PersistencePolicies.APPLICATION,
+ RMRegistryOperationsService.PurgePolicy.FailOnChildren,
+ null));
+ // succeed for container
+ assertEquals(1, registry.purgeRecords("/", cid2,
+ PersistencePolicies.CONTAINER,
+ RMRegistryOperationsService.PurgePolicy.FailOnChildren,
+ null));
+ assertPathNotFound(dns2path);
+ assertPathExists(dns1path);
+
+ // attempt to delete root with policy of fail on children
+ try {
+ registry.purgeRecords("/",
+ appId,
+ PersistencePolicies.APPLICATION,
+ RMRegistryOperationsService.PurgePolicy.FailOnChildren, null);
+ fail("expected a failure");
+ } catch (PathIsNotEmptyDirectoryException expected) {
+ // expected
+ }
+ assertPathExists(appPath);
+ assertPathExists(dns1path);
+
+ // downgrade to a skip on children
+ assertEquals(0,
+ registry.purgeRecords("/", appId,
+ PersistencePolicies.APPLICATION,
+ RMRegistryOperationsService.PurgePolicy.SkipOnChildren,
+ null));
+ assertPathExists(appPath);
+ assertPathExists(dns1path);
+
+ // now trigger recursive delete
+ assertEquals(1,
+ registry.purgeRecords("/",
+ appId,
+ PersistencePolicies.APPLICATION,
+ RMRegistryOperationsService.PurgePolicy.PurgeAll,
+ null));
+ assertPathNotFound(appPath);
+ assertPathNotFound(dns1path);
+
+ }
+
+
+ @Test
+ public void testAsyncPurgeEntry() throws Throwable {
+
+ String path = "/users/example/hbase/hbase1/";
+ ServiceRecord written = buildExampleServiceEntry(
+ PersistencePolicies.APPLICATION_ATTEMPT);
+ written.id = "testAsyncPurgeEntry_attempt_001";
+
+ operations.mkdir(RegistryPathUtils.parentOf(path), true);
+ operations.create(path, written, 0);
+
+ ZKPathDumper dump = registry.dumpPath();
+ CuratorEventCatcher events = new CuratorEventCatcher();
+
+ LOG.info("Initial state {}", dump);
+
+ // container query
+ int opcount = registry.purgeRecords("/",
+ written.id,
+ PersistencePolicies.CONTAINER,
+ RMRegistryOperationsService.PurgePolicy.PurgeAll,
+ events);
+ assertPathExists(path);
+ assertEquals(0, opcount);
+ assertEquals("Event counter", 0, events.getCount());
+
+
+ // now the application attempt
+ opcount = registry.purgeRecords("/",
+ written.id,
+ -1,
+// PersistencePolicies.APPLICATION_ATTEMPT,
+ RMRegistryOperationsService.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 testPutGetEphemeralServiceEntry() throws Throwable {
+
+ String path = ENTRY_PATH;
+ ServiceRecord written = buildExampleServiceEntry(
+ PersistencePolicies.EPHEMERAL);
+
+ operations.mkdir(RegistryPathUtils.parentOf(path), true);
+ operations.create(path, written, CreateFlags.EPHEMERAL);
+ ServiceRecord resolved = operations.resolve(path);
+ validateEntry(resolved);
+ assertMatches(written, resolved);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testPutGetEphemeralServiceEntryWrongPolicy() throws Throwable {
+ String path = ENTRY_PATH;
+ ServiceRecord written = buildExampleServiceEntry(
+ PersistencePolicies.APPLICATION_ATTEMPT);
+
+ operations.mkdir(RegistryPathUtils.parentOf(path), true);
+ operations.create(path, written, CreateFlags.EPHEMERAL);
+ }
+
+ @Test
+ public void testEphemeralNoChildren() throws Throwable {
+ ServiceRecord webapp = new ServiceRecord("1",
+ "tomcat-based web application", PersistencePolicies.EPHEMERAL);
+ operations.mkdir(USERPATH, false);
+ String appPath = USERPATH + "tomcat2";
+
+ operations.create(appPath, webapp, CreateFlags.EPHEMERAL);
+ String components = appPath + RegistryConstants.SUBPATH_COMPONENTS + "/";
+ try {
+ operations.mkdir(components, false);
+ fail("expected an error");
+ } catch (NoChildrenForEphemeralsException expected) {
+ // expected
+ }
+ try {
+ operations.create(appPath + "/subdir", webapp, CreateFlags.EPHEMERAL);
+ fail("expected an error");
+ } catch (NoChildrenForEphemeralsException expected) {
+ // expected
+ }
+
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testPolicyConflict() throws Throwable {
+ ServiceRecord rec = buildExampleServiceEntry(PersistencePolicies.EPHEMERAL);
+ operations.mkdir(USERPATH, false);
+ String appPath = USERPATH + "ex";
+ operations.create(appPath, rec, 0);
+ }
+
+}
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..78fa658
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/resources/log4j.properties
@@ -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.
+
+# 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
+#log4j.logger.org.apache.slider.yarn.appmaster.SliderAppMasterer.master=INFO,subprocess
+
+# packages under test
+org.apache.hadoop.yarn.registry=DEBUG
+org.apache.hadoop.service=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=WARN
+log4j.logger.org.apache.zookeeper.ClientCnxn=FATAL
+
+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
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
index acf330f..6c04f17 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
@@ -59,6 +59,11 @@
+ org.apache.hadoop
+ hadoop-yarn-registry
+
+
+ com.google.guavaguava
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index 01d5064..553f7dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -27,6 +27,7 @@
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.registry.RMRegistryService;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
@@ -103,4 +104,6 @@ void setRMApplicationHistoryWriter(
boolean isWorkPreservingRecoveryEnabled();
int getEpoch();
+
+ RMRegistryService getRegistry();
}
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index f72ef30..c95ebea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -32,6 +32,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.registry.RMRegistryService;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
@@ -83,6 +84,7 @@
private RMApplicationHistoryWriter rmApplicationHistoryWriter;
private ConfigurationProvider configurationProvider;
private int epoch;
+ private RMRegistryService registry;
/**
* Default constructor. To be used in conjunction with setter methods for
@@ -103,7 +105,8 @@ public RMContextImpl(Dispatcher rmDispatcher,
RMContainerTokenSecretManager containerTokenSecretManager,
NMTokenSecretManagerInRM nmTokenSecretManager,
ClientToAMTokenSecretManagerInRM clientToAMTokenSecretManager,
- RMApplicationHistoryWriter rmApplicationHistoryWriter) {
+ RMApplicationHistoryWriter rmApplicationHistoryWriter,
+ RMRegistryService registry) {
this();
this.setDispatcher(rmDispatcher);
this.setContainerAllocationExpirer(containerAllocationExpirer);
@@ -115,6 +118,7 @@ public RMContextImpl(Dispatcher rmDispatcher,
this.setNMTokenSecretManager(nmTokenSecretManager);
this.setClientToAMTokenSecretManager(clientToAMTokenSecretManager);
this.setRMApplicationHistoryWriter(rmApplicationHistoryWriter);
+ this.setRegistry(registry);
RMStateStore nullStore = new NullRMStateStore();
nullStore.setRMDispatcher(rmDispatcher);
@@ -369,4 +373,13 @@ public int getEpoch() {
void setEpoch(int epoch) {
this.epoch = epoch;
}
+
+ @Override
+ public RMRegistryService getRegistry() {
+ return registry;
+ }
+
+ void setRegistry(RMRegistryService registry) {
+ this.registry = registry;
+ }
}
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index f315702..addc984 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -61,6 +61,7 @@
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
@@ -71,6 +72,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreFactory;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
+import org.apache.hadoop.yarn.server.resourcemanager.registry.RMRegistryService;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
@@ -157,6 +159,11 @@
private AppReportFetcher fetcher = null;
protected ResourceTrackerService resourceTracker;
+ /**
+ * Registry service
+ */
+ protected RMRegistryService registry;
+
@VisibleForTesting
protected String webAppAddress;
private ConfigurationProvider configurationProvider = null;
@@ -231,6 +238,15 @@ protected void serviceInit(Configuration conf) throws Exception {
if (this.rmContext.isHAEnabled()) {
HAUtil.verifyAndSetConfiguration(this.conf);
}
+
+ boolean registryEnabled = conf.getBoolean(RegistryConstants.KEY_REGISTRY_ENABLED,
+ RegistryConstants.DEFAULT_REGISTRY_ENABLED);
+ if (registryEnabled) {
+ registry = new RMRegistryService(rmContext);
+ addService(registry);
+ rmContext.setRegistry(registry);
+ }
+
createAndInitActiveServices();
webAppAddress = WebAppUtils.getWebAppBindURL(this.conf,
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/registry/RMRegistryService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/registry/RMRegistryService.java
new file mode 100644
index 0000000..7ce49a5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/registry/RMRegistryService.java
@@ -0,0 +1,257 @@
+/*
+ * 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.yarn.server.resourcemanager.registry;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+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.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.registry.server.services.RMRegistryOperationsService;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreAppEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * This is the RM service which translates from RM events
+ * to registry actions
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RMRegistryService extends CompositeService {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(RMRegistryService.class);
+
+ private final RMContext rmContext;
+
+ /**
+ * Registry service
+ */
+ private final RMRegistryOperationsService registryOperations;
+
+ public RMRegistryService(RMContext rmContext) {
+ super(RMRegistryService.class.getName());
+ this.rmContext = rmContext;
+ registryOperations =
+ new RMRegistryOperationsService("Registry");
+ addService(registryOperations);
+ }
+
+
+ @Override
+ protected void serviceStart() throws Exception {
+ super.serviceStart();
+ LOG.info("RM registry service started : {}",
+ registryOperations.bindingDiagnosticDetails());
+ // Register self as event handler for RM Events
+ register(RMAppAttemptEventType.class, new AppEventHandler());
+ register(RMAppManagerEventType.class, new AppManagerEventHandler());
+ register(RMStateStoreEventType.class, new StateStoreEventHandler());
+ register(RMContainerEventType.class, new ContainerEventHandler());
+ }
+
+ /**
+ * register a handler
+ * @param eventType event type
+ * @param handler handler
+ */
+ private void register(Class extends Enum> eventType,
+ EventHandler handler) {
+ rmContext.getDispatcher().register(eventType, handler);
+ }
+
+
+ @SuppressWarnings(
+ {"EnumSwitchStatementWhichMissesCases", "UnnecessaryDefault"})
+ protected void handleAppManagerEvent(RMAppManagerEvent event) throws
+ IOException {
+ RMAppManagerEventType eventType = event.getType();
+ ApplicationId appId =
+ event.getApplicationId();
+ switch (eventType) {
+ case APP_COMPLETED:
+ registryOperations.onApplicationCompleted(appId);
+ break;
+ default:
+ // this isn't in the enum today...just making sure for the
+ // future
+ break;
+ }
+ }
+
+ @SuppressWarnings("EnumSwitchStatementWhichMissesCases")
+ private void handleStateStoreEvent(RMStateStoreEvent event)
+ throws IOException {
+ RMStateStoreEventType eventType = event.getType();
+ switch (eventType) {
+ case STORE_APP:
+ RMStateStoreAppEvent storeAppEvent = (RMStateStoreAppEvent) event;
+ RMStateStore.ApplicationState appState = storeAppEvent.getAppState();
+ ApplicationId appId = appState.getAppId();
+ registryOperations.onStateStoreEvent(appId, appState.getUser());
+ break;
+
+ default:
+ break;
+ }
+ }
+
+
+ @SuppressWarnings("EnumSwitchStatementWhichMissesCases")
+ protected void handleAppAttemptEvent(RMAppAttemptEvent event) throws
+ IOException {
+ RMAppAttemptEventType eventType = event.getType();
+ ApplicationAttemptId appAttemptId =
+ event.getApplicationAttemptId();
+ ApplicationId appId = appAttemptId.getApplicationId();
+ switch (eventType) {
+
+ case REGISTERED:
+ RMAppAttemptRegistrationEvent evt =
+ (RMAppAttemptRegistrationEvent) event;
+ registryOperations.onApplicationAttemptRegistered(appAttemptId,
+ evt.getHost(),
+ evt.getRpcport(),
+ evt.getTrackingurl());
+ break;
+
+ case UNREGISTERED:
+ registryOperations.onApplicationUnregistered(appAttemptId);
+ break;
+
+ case ATTEMPT_ADDED:
+ registryOperations.onApplicationAttemptAdded(appAttemptId);
+ break;
+
+ // container has finished
+ case CONTAINER_FINISHED:
+ RMAppAttemptContainerFinishedEvent cfe =
+ (RMAppAttemptContainerFinishedEvent) event;
+ ContainerId containerId = cfe.getContainerStatus().getContainerId();
+ registryOperations.onAMContainerFinished(containerId);
+ break;
+
+
+ default:
+ // do nothing
+ }
+ }
+
+ @SuppressWarnings("EnumSwitchStatementWhichMissesCases")
+ private void handleContainerEvent(RMContainerEvent event)
+ throws IOException {
+ RMContainerEventType eventType = event.getType();
+ switch (eventType) {
+ case FINISHED:
+ ContainerId containerId = event.getContainerId();
+ registryOperations.onContainerFinished(containerId);
+
+ default:
+ break;
+ }
+ }
+
+
+ /**
+ * Handler for app events
+ */
+ private class AppEventHandler implements
+ EventHandler {
+
+ /**
+ * Handle an application event
+ * @param event
+ */
+ @Override
+ public void handle(RMAppAttemptEvent event) {
+ try {
+ handleAppAttemptEvent(event);
+ } catch (IOException e) {
+ LOG.warn("handling {}: {}", event, e, e);
+ }
+ }
+ }
+
+ /**
+ * Handler for RM-side App manager events
+ */
+
+ private class AppManagerEventHandler
+ implements EventHandler {
+ @Override
+ public void handle(RMAppManagerEvent event) {
+ try {
+ handleAppManagerEvent(event);
+ } catch (IOException e) {
+ LOG.warn("handling {}: {}", event, e, e);
+ }
+ }
+ }
+
+ /**
+ * Handler for RM-side state store events.
+ * This happens early on, and as the data contains the user details,
+ * it is where paths can be set up in advance of being used.
+ */
+
+ private class StateStoreEventHandler implements EventHandler {
+ @Override
+ public void handle(RMStateStoreEvent event) {
+ try {
+ handleStateStoreEvent(event);
+ } catch (IOException e) {
+ LOG.warn("handling {}: {}", event, e, e);
+ }
+ }
+ }
+
+ /**
+ * Handler for RM-side container events
+ */
+ private class ContainerEventHandler implements EventHandler {
+
+ @Override
+ public void handle(RMContainerEvent event) {
+ try {
+ handleContainerEvent(event);
+ } catch (IOException e) {
+ LOG.warn("handling {}: {}", event, e, e);
+ }
+ }
+ }
+
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
index d720eb6..d2007dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
@@ -112,7 +112,7 @@ public static RMContext mockRMContext(int n, long time) {
RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
RMContext context = new RMContextImpl(rmDispatcher,
containerAllocationExpirer, amLivelinessMonitor, amFinishingMonitor,
- null, null, null, null, null, writer) {
+ null, null, null, null, null, writer, null) {
@Override
public ConcurrentMap getRMApps() {
return map;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
index d877e25..e831a7a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
@@ -105,7 +105,8 @@ public void setUp() throws Exception {
rmContext =
new RMContextImpl(rmDispatcher, null, null, null,
- mock(DelegationTokenRenewer.class), null, null, null, null, null);
+ mock(DelegationTokenRenewer.class), null, null, null, null, null,
+ null);
NodesListManager nodesListManager = mock(NodesListManager.class);
HostsFileReader reader = mock(HostsFileReader.class);
when(nodesListManager.getHostsReader()).thenReturn(reader);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java
index c837450..3f68990 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java
@@ -72,7 +72,7 @@ public void setUp() {
// Dispatcher that processes events inline
Dispatcher dispatcher = new InlineDispatcher();
RMContext context = new RMContextImpl(dispatcher, null,
- null, null, null, null, null, null, null, null);
+ null, null, null, null, null, null, null, null, null);
dispatcher.register(SchedulerEventType.class,
new InlineDispatcher.EmptyEventHandler());
dispatcher.register(RMNodeEventType.class,
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java
index d16d551..1b03565 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java
@@ -74,7 +74,7 @@ public void setUp() {
new TestRMNodeEventDispatcher());
RMContext context = new RMContextImpl(dispatcher, null,
- null, null, null, null, null, null, null, null);
+ null, null, null, null, null, null, null, null, null);
dispatcher.register(SchedulerEventType.class,
new InlineDispatcher.EmptyEventHandler());
dispatcher.register(RMNodeEventType.class,
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java
index 4f94695..584a2e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java
@@ -71,7 +71,7 @@ public void handle(Event event) {
RMContext context =
new RMContextImpl(dispatcher, null, null, null, null,
null, new RMContainerTokenSecretManager(conf),
- new NMTokenSecretManagerInRM(conf), null, null);
+ new NMTokenSecretManagerInRM(conf), null, null, null);
dispatcher.register(RMNodeEventType.class,
new ResourceManager.NodeEventDispatcher(context));
NodesListManager nodesListManager = new NodesListManager(context);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
index 2fc4431..bb89db6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
@@ -196,7 +196,7 @@ public void setUp() throws Exception {
new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM(),
- writer);
+ writer, null);
((RMContextImpl)rmContext).setStateStore(store);
rmDispatcher.register(RMAppAttemptEventType.class,
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
index efcecd9..40dff92 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
@@ -234,7 +234,7 @@ public void setUp() throws Exception {
new RMContainerTokenSecretManager(conf),
nmTokenManager,
clientToAMTokenManager,
- writer);
+ writer, null);
store = mock(RMStateStore.class);
((RMContextImpl) rmContext).setStateStore(store);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index e029749..14d250a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -392,7 +392,7 @@ public void testRefreshQueues() throws Exception {
RMContextImpl rmContext = new RMContextImpl(null, null, null, null, null,
null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
- new ClientToAMTokenSecretManagerInRM(), null);
+ new ClientToAMTokenSecretManagerInRM(), null, null);
setupQueueConfiguration(conf);
cs.setConf(new YarnConfiguration());
cs.setRMContext(resourceManager.getRMContext());
@@ -498,7 +498,7 @@ public void testParseQueue() throws IOException {
cs.reinitialize(conf, new RMContextImpl(null, null, null, null, null,
null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
- new ClientToAMTokenSecretManagerInRM(), null));
+ new ClientToAMTokenSecretManagerInRM(), null, null));
}
@Test
@@ -514,7 +514,7 @@ public void testReconnectedNode() throws Exception {
cs.reinitialize(csConf, new RMContextImpl(null, null, null, null,
null, null, new RMContainerTokenSecretManager(csConf),
new NMTokenSecretManagerInRM(csConf),
- new ClientToAMTokenSecretManagerInRM(), null));
+ new ClientToAMTokenSecretManagerInRM(), null, null));
RMNode n1 = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1);
RMNode n2 = MockNodes.newNodeInfo(0, MockNodes.newResource(2 * GB), 2);
@@ -545,7 +545,7 @@ public void testRefreshQueuesWithNewQueue() throws Exception {
cs.reinitialize(conf, new RMContextImpl(null, null, null, null, null,
null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
- new ClientToAMTokenSecretManagerInRM(), null));
+ new ClientToAMTokenSecretManagerInRM(), null, null));
checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
// Add a new queue b4
@@ -963,7 +963,7 @@ public void testNumClusterNodes() throws Exception {
RMContextImpl rmContext = new RMContextImpl(null, null, null, null, null,
null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
- new ClientToAMTokenSecretManagerInRM(), null);
+ new ClientToAMTokenSecretManagerInRM(), null, null);
cs.setRMContext(rmContext);
CapacitySchedulerConfiguration csConf =
new CapacitySchedulerConfiguration();
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueMappings.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueMappings.java
index f573f43..12ab5c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueMappings.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueMappings.java
@@ -82,7 +82,7 @@ public void testQueueMapping() throws Exception {
RMContextImpl rmContext = new RMContextImpl(null, null, null, null, null,
null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
- new ClientToAMTokenSecretManagerInRM(), null);
+ new ClientToAMTokenSecretManagerInRM(), null, null);
cs.setConf(conf);
cs.setRMContext(rmContext);
cs.init(conf);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java
index a3b990c..a0e67ad 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java
@@ -46,7 +46,7 @@ public void testQueueParsing() throws Exception {
RMContextImpl rmContext = new RMContextImpl(null, null,
null, null, null, null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
- new ClientToAMTokenSecretManagerInRM(), null);
+ new ClientToAMTokenSecretManagerInRM(), null, null);
capacityScheduler.setConf(conf);
capacityScheduler.setRMContext(rmContext);
capacityScheduler.init(conf);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
index e548661..504ec37 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
@@ -91,7 +91,7 @@ public EventHandler getEventHandler() {
new AMRMTokenSecretManager(conf, null),
new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
- new ClientToAMTokenSecretManagerInRM(), writer);
+ new ClientToAMTokenSecretManagerInRM(), writer, null);
return rmContext;
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
index 3d38364..27a6fd0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
@@ -144,7 +144,7 @@ public void testAppAttemptMetrics() throws Exception {
AsyncDispatcher dispatcher = new InlineDispatcher();
RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
RMContext rmContext = new RMContextImpl(dispatcher, null,
- null, null, null, null, null, null, null, writer);
+ null, null, null, null, null, null, null, writer, null);
FifoScheduler scheduler = new FifoScheduler();
Configuration conf = new Configuration();
@@ -187,7 +187,8 @@ public void testNodeLocalAssignment() throws Exception {
nmTokenSecretManager.rollMasterKey();
RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null,
- null, containerTokenSecretManager, nmTokenSecretManager, null, writer);
+ null, containerTokenSecretManager, nmTokenSecretManager, null, writer,
+ null);
FifoScheduler scheduler = new FifoScheduler();
scheduler.setRMContext(rmContext);
@@ -256,7 +257,8 @@ public void testUpdateResourceOnNode() throws Exception {
nmTokenSecretManager.rollMasterKey();
RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null,
- null, containerTokenSecretManager, nmTokenSecretManager, null, writer);
+ null, containerTokenSecretManager, nmTokenSecretManager, null, writer,
+ null);
FifoScheduler scheduler = new FifoScheduler(){
@SuppressWarnings("unused")
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
index 0df7c0d..4e0c810 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
@@ -163,7 +163,7 @@ public static RMContext mockRMContext(int numApps, int racks, int numNodes,
deactivatedNodesMap.put(node.getHostName(), node);
}
return new RMContextImpl(null, null, null, null,
- null, null, null, null, null, null) {
+ null, null, null, null, null, null, null) {
@Override
public ConcurrentMap getRMApps() {
return applicationsMaps;
@@ -206,7 +206,7 @@ public static CapacityScheduler mockCapacityScheduler() throws IOException {
cs.setRMContext(new RMContextImpl(null, null, null, null, null,
null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
- new ClientToAMTokenSecretManagerInRM(), null));
+ new ClientToAMTokenSecretManagerInRM(), null, null));
cs.init(conf);
return cs;
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
index 111bf47..25e1683 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
@@ -150,7 +150,7 @@ public YarnApplicationState createApplicationState() {
}
return new RMContextImpl(null, null, null, null,
- null, null, null, null, null, null) {
+ null, null, null, null, null, null, null) {
@Override
public ConcurrentMap getRMApps() {
return applicationsMaps;
@@ -173,7 +173,7 @@ private static FairScheduler mockFairScheduler() throws IOException {
fs.setRMContext(new RMContextImpl(null, null, null, null, null,
null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
- new ClientToAMTokenSecretManagerInRM(), null));
+ new ClientToAMTokenSecretManagerInRM(), null, null));
fs.init(conf);
return fs;
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
index b5b7a43..6b2352b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
@@ -49,6 +49,8 @@
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.registry.server.services.MicroZookeeperService;
import org.apache.hadoop.yarn.server.api.ResourceTracker;
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
@@ -106,7 +108,7 @@
private String[] rmIds;
private ApplicationHistoryServer appHistoryServer;
-
+ private MicroZookeeperService zookeeper;
private boolean useFixedPorts;
private boolean useRpc = false;
private int failoverTimeout;
@@ -121,6 +123,7 @@
// Number of nm-log-dirs per nodemanager
private int numLogDirs;
private boolean enableAHS;
+ private final boolean enableRegistry;
/**
* @param testName name of the test
@@ -129,14 +132,17 @@
* @param numLocalDirs the number of nm-local-dirs per nodemanager
* @param numLogDirs the number of nm-log-dirs per nodemanager
* @param enableAHS enable ApplicationHistoryServer or not
+ * @param enableRegistry enable the registry or not
*/
public MiniYARNCluster(
String testName, int numResourceManagers, int numNodeManagers,
- int numLocalDirs, int numLogDirs, boolean enableAHS) {
+ int numLocalDirs, int numLogDirs, boolean enableAHS,
+ boolean enableRegistry) {
super(testName.replace("$", ""));
this.numLocalDirs = numLocalDirs;
this.numLogDirs = numLogDirs;
this.enableAHS = enableAHS;
+ this.enableRegistry = enableRegistry;
String testSubDir = testName.replace("$", "");
File targetWorkDir = new File("target", testSubDir);
try {
@@ -192,6 +198,21 @@ public MiniYARNCluster(
* @param numNodeManagers the number of node managers in the cluster
* @param numLocalDirs the number of nm-local-dirs per nodemanager
* @param numLogDirs the number of nm-log-dirs per nodemanager
+ * @param enableAHS enable ApplicationHistoryServer or not
+ */
+ public MiniYARNCluster(
+ String testName, int numResourceManagers, int numNodeManagers,
+ int numLocalDirs, int numLogDirs, boolean enableAHS) {
+ this(testName, numResourceManagers, numNodeManagers, numLocalDirs,
+ numLogDirs, enableAHS, false);
+ }
+
+ /**
+ * @param testName name of the test
+ * @param numResourceManagers the number of resource managers in the cluster
+ * @param numNodeManagers the number of node managers in the cluster
+ * @param numLocalDirs the number of nm-local-dirs per nodemanager
+ * @param numLogDirs the number of nm-log-dirs per nodemanager
*/
public MiniYARNCluster(
String testName, int numResourceManagers, int numNodeManagers,
@@ -243,6 +264,15 @@ public void serviceInit(Configuration conf) throws Exception {
rmIds = rmIdsCollection.toArray(new String[rmIdsCollection.size()]);
}
+ // enable the in-memory ZK cluster AHEAD of RMs to ensure it starts first
+ if (enableRegistry) {
+ zookeeper = new MicroZookeeperService("Local ZK service");
+ addService(zookeeper);
+ conf.setBooleanIfUnset(RegistryConstants.KEY_REGISTRY_ENABLED, true);
+ conf.set(RegistryConstants.KEY_ZKSERVICE_DATADIR,
+ new File(testWorkDir, "zookeeper").getAbsolutePath());
+ }
+
for (int i = 0; i < resourceManagers.length; i++) {
resourceManagers[i] = createResourceManager();
if (!useFixedPorts) {
@@ -742,4 +772,8 @@ protected void doSecureLogin() throws IOException {
public int getNumOfResourceManager() {
return this.resourceManagers.length;
}
+
+ public MicroZookeeperService getZookeeper() {
+ return zookeeper;
+ }
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYARNClusterRegistry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYARNClusterRegistry.java
new file mode 100644
index 0000000..2baae7a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYARNClusterRegistry.java
@@ -0,0 +1,111 @@
+/*
+ * 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.yarn.server;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.registry.client.services.RegistryOperationsService;
+import org.apache.hadoop.yarn.registry.server.services.MicroZookeeperService;
+import org.apache.hadoop.yarn.registry.server.services.RMRegistryOperationsService;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.registry.RMRegistryService;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+/**
+ * Test registry support in the cluster
+ */
+public class TestMiniYARNClusterRegistry extends Assert {
+
+
+ MiniYARNCluster cluster;
+
+ @Rule
+ public final Timeout testTimeout = new Timeout(10000);
+
+ @Rule
+ public TestName methodName = new TestName();
+ private Configuration conf;
+
+ @Before
+ public void setup() throws IOException, InterruptedException {
+ conf = new YarnConfiguration();
+
+ cluster = new MiniYARNCluster(methodName.getMethodName(),
+ 1, 1, 1, 1, false, true);
+ cluster.init(conf);
+ cluster.start();
+ }
+
+ @Test
+ public void testZKInstance() throws Exception {
+ assertNotNull("zookeeper", cluster.getZookeeper());
+ }
+
+ @Test
+ public void testZKConnectionAddress() throws Exception {
+ MicroZookeeperService zookeeper = cluster.getZookeeper();
+ InetSocketAddress address = zookeeper.getConnectionAddress();
+ assertTrue("Unconfigured address", address.getPort() != 0);
+ }
+
+ @Test
+ public void testZKConfigPatchPropagaton() throws Exception {
+ MicroZookeeperService zookeeper = cluster.getZookeeper();
+ String connectionString = zookeeper.getConnectionString();
+ String confConnection = conf.get(RegistryConstants.KEY_REGISTRY_ZK_QUORUM);
+ assertNotNull(confConnection);
+ assertEquals(connectionString, confConnection);
+ }
+
+ @Test
+ public void testRegistryCreated() throws Exception {
+ assertTrue("registry not enabled",
+ conf.getBoolean(RegistryConstants.KEY_REGISTRY_ENABLED, false));
+ MicroZookeeperService zookeeper = cluster.getZookeeper();
+ String connectionString = zookeeper.getConnectionString();
+ String confConnection = conf.get(RegistryConstants.KEY_REGISTRY_ZK_QUORUM);
+ ResourceManager rm = cluster.getResourceManager(0);
+ RMRegistryService registry = rm.getRMContext().getRegistry();
+ assertNotNull("null registry", registry);
+ }
+
+ @Test
+ public void testPathsExist() throws Throwable {
+ MicroZookeeperService zookeeper = cluster.getZookeeper();
+ // service to directly hook in to the ZK server
+ RegistryOperationsService operations =
+ new RegistryOperationsService("operations", zookeeper);
+ operations.init(new YarnConfiguration());
+ operations.start();
+
+ operations.stat("/");
+ //verifies that the RM startup has created the system services path
+ operations.stat(RegistryConstants.PATH_SYSTEM_SERVICES);
+
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/pom.xml b/hadoop-yarn-project/hadoop-yarn/pom.xml
index f6479bc..e258f21 100644
--- a/hadoop-yarn-project/hadoop-yarn/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/pom.xml
@@ -100,5 +100,6 @@
hadoop-yarn-applicationshadoop-yarn-sitehadoop-yarn-client
+ hadoop-yarn-registry